2017-05-26 04:48:44 +08:00
|
|
|
/*
|
|
|
|
* DatabaseBackupAgent.actor.cpp
|
|
|
|
*
|
|
|
|
* This source file is part of the FoundationDB open source project
|
|
|
|
*
|
2022-03-22 04:36:23 +08:00
|
|
|
* Copyright 2013-2022 Apple Inc. and the FoundationDB project authors
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* 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
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* http://www.apache.org/licenses/LICENSE-2.0
|
2018-02-22 02:25:11 +08:00
|
|
|
*
|
2017-05-26 04:48:44 +08:00
|
|
|
* 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.
|
|
|
|
*/
|
|
|
|
|
2019-03-02 06:49:04 +08:00
|
|
|
#include <iterator>
|
2019-02-18 07:19:05 +08:00
|
|
|
#include "fdbclient/BackupAgent.actor.h"
|
2019-03-02 06:49:04 +08:00
|
|
|
#include "fdbclient/Status.h"
|
|
|
|
#include "fdbclient/StatusClient.h"
|
|
|
|
#include "fdbclient/DatabaseContext.h"
|
|
|
|
#include "fdbclient/NativeAPI.actor.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include <ctime>
|
|
|
|
#include <climits>
|
2022-06-24 07:03:53 +08:00
|
|
|
#include "flow/IAsyncFile.h"
|
2017-05-26 04:48:44 +08:00
|
|
|
#include "flow/genericactors.actor.h"
|
|
|
|
#include "flow/Hash3.h"
|
|
|
|
#include <numeric>
|
2019-02-18 09:38:13 +08:00
|
|
|
#include "fdbclient/ManagementAPI.actor.h"
|
2018-10-20 01:30:13 +08:00
|
|
|
#include "fdbclient/KeyBackedTypes.h"
|
2020-06-16 18:59:47 +08:00
|
|
|
#include <inttypes.h>
|
|
|
|
#include <map>
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2022-03-16 15:08:59 +08:00
|
|
|
#include "flow/actorcompiler.h" // has to be last include
|
|
|
|
|
2022-09-20 02:35:58 +08:00
|
|
|
const Key DatabaseBackupAgent::keyAddPrefix = "add_prefix"_sr;
|
|
|
|
const Key DatabaseBackupAgent::keyRemovePrefix = "remove_prefix"_sr;
|
|
|
|
const Key DatabaseBackupAgent::keyRangeVersions = "range_versions"_sr;
|
|
|
|
const Key DatabaseBackupAgent::keyCopyStop = "copy_stop"_sr;
|
|
|
|
const Key DatabaseBackupAgent::keyDatabasesInSync = "databases_in_sync"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
const int DatabaseBackupAgent::LATEST_DR_VERSION = 1;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DatabaseBackupAgent::DatabaseBackupAgent()
|
2021-07-25 02:20:51 +08:00
|
|
|
: subspace(Subspace(databaseBackupPrefixRange.begin)), states(subspace.get(BackupAgentBase::keyStates)),
|
|
|
|
config(subspace.get(BackupAgentBase::keyConfig)), errors(subspace.get(BackupAgentBase::keyErrors)),
|
|
|
|
ranges(subspace.get(BackupAgentBase::keyRanges)), tagNames(subspace.get(BackupAgentBase::keyTagName)),
|
|
|
|
sourceStates(subspace.get(BackupAgentBase::keySourceStates)),
|
|
|
|
sourceTagNames(subspace.get(BackupAgentBase::keyTagName)),
|
2021-07-05 04:14:25 +08:00
|
|
|
taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks),
|
2021-07-17 15:11:40 +08:00
|
|
|
AccessSystemKeys::True,
|
|
|
|
PriorityBatch::False,
|
|
|
|
LockAware::True)),
|
2021-07-25 02:20:51 +08:00
|
|
|
futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)) {
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
DatabaseBackupAgent::DatabaseBackupAgent(Database src)
|
2021-07-25 02:20:51 +08:00
|
|
|
: subspace(Subspace(databaseBackupPrefixRange.begin)), states(subspace.get(BackupAgentBase::keyStates)),
|
|
|
|
config(subspace.get(BackupAgentBase::keyConfig)), errors(subspace.get(BackupAgentBase::keyErrors)),
|
|
|
|
ranges(subspace.get(BackupAgentBase::keyRanges)), tagNames(subspace.get(BackupAgentBase::keyTagName)),
|
|
|
|
sourceStates(subspace.get(BackupAgentBase::keySourceStates)),
|
|
|
|
sourceTagNames(subspace.get(BackupAgentBase::keyTagName)),
|
2021-07-05 04:14:25 +08:00
|
|
|
taskBucket(new TaskBucket(subspace.get(BackupAgentBase::keyTasks),
|
2021-07-17 15:11:40 +08:00
|
|
|
AccessSystemKeys::True,
|
|
|
|
PriorityBatch::False,
|
|
|
|
LockAware::True)),
|
2021-07-25 02:20:51 +08:00
|
|
|
futureBucket(new FutureBucket(subspace.get(BackupAgentBase::keyFutures), AccessSystemKeys::True, LockAware::True)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket->src = src;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Any new per-DR properties should go here.
|
|
|
|
class DRConfig {
|
|
|
|
public:
|
|
|
|
DRConfig(UID uid = UID())
|
2022-09-20 02:35:58 +08:00
|
|
|
: uid(uid), configSpace(uidPrefixKey("uid->config/"_sr.withPrefix(databaseBackupPrefixRange.begin), uid)) {}
|
2017-05-26 04:48:44 +08:00
|
|
|
DRConfig(Reference<Task> task)
|
|
|
|
: DRConfig(BinaryReader::fromStringRef<UID>(task->params[BackupAgentBase::keyConfigLogUid], Unversioned())) {}
|
|
|
|
|
2022-10-01 07:05:47 +08:00
|
|
|
KeyBackedBinaryValue<int64_t> rangeBytesWritten() { return configSpace.pack(__FUNCTION__sr); }
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2022-10-01 07:05:47 +08:00
|
|
|
KeyBackedBinaryValue<int64_t> logBytesWritten() { return configSpace.pack(__FUNCTION__sr); }
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
void clear(Reference<ReadYourWritesTransaction> tr) { tr->clear(configSpace.range()); }
|
|
|
|
|
|
|
|
UID getUid() { return uid; }
|
|
|
|
|
|
|
|
private:
|
|
|
|
UID uid;
|
|
|
|
Subspace configSpace;
|
|
|
|
};
|
|
|
|
|
|
|
|
namespace dbBackup {
|
|
|
|
|
|
|
|
bool copyDefaultParameters(Reference<Task> source, Reference<Task> dest) {
|
|
|
|
if (source) {
|
|
|
|
copyParameter(source, dest, BackupAgentBase::keyFolderId);
|
|
|
|
copyParameter(source, dest, BackupAgentBase::keyConfigLogUid);
|
2018-02-21 05:22:31 +08:00
|
|
|
copyParameter(source, dest, BackupAgentBase::destUid);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
copyParameter(source, dest, DatabaseBackupAgent::keyAddPrefix);
|
|
|
|
copyParameter(source, dest, DatabaseBackupAgent::keyRemovePrefix);
|
|
|
|
return true;
|
|
|
|
}
|
|
|
|
|
|
|
|
return false;
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR template <class Tr>
|
|
|
|
Future<Void> checkTaskVersion(Tr tr, Reference<Task> task, StringRef name, uint32_t version) {
|
|
|
|
uint32_t taskVersion = task->getVersion();
|
|
|
|
if (taskVersion > version) {
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent(SevError, "BA_BackupRangeTaskFuncExecute")
|
|
|
|
.detail("TaskVersion", taskVersion)
|
|
|
|
.detail("Name", name)
|
|
|
|
.detail("Version", version);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(logError(tr,
|
|
|
|
Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyErrors)
|
|
|
|
.pack(task->params[BackupAgentBase::keyConfigLogUid]),
|
2017-05-26 04:48:44 +08:00
|
|
|
format("ERROR: %s task version `%lu' is greater than supported version `%lu'",
|
|
|
|
task->params[Task::reservedTaskParamKeyType].toString().c_str(),
|
|
|
|
(unsigned long)taskVersion,
|
|
|
|
(unsigned long)version)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
throw task_invalid_version();
|
|
|
|
}
|
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
struct BackupRangeTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
|
|
|
static constexpr uint32_t version = 1;
|
|
|
|
|
|
|
|
static struct {
|
2022-10-01 07:05:47 +08:00
|
|
|
static TaskParam<int64_t> bytesWritten() { return __FUNCTION__sr; }
|
2017-05-26 04:48:44 +08:00
|
|
|
} Params;
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
static const Key keyAddBackupRangeTasks;
|
|
|
|
static const Key keyBackupRangeBeginKey;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
2021-01-26 10:03:09 +08:00
|
|
|
return _execute(cx, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2017-05-26 04:48:44 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Standalone<VectorRef<KeyRef>>> getBlockOfShards(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Key beginKey,
|
|
|
|
Key endKey,
|
|
|
|
int limit) {
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2020-06-16 18:59:47 +08:00
|
|
|
state Standalone<VectorRef<KeyRef>> results;
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult values = wait(tr->getRange(
|
2017-05-26 04:48:44 +08:00
|
|
|
KeyRangeRef(keyAfter(beginKey.withPrefix(keyServersPrefix)), endKey.withPrefix(keyServersPrefix)), limit));
|
|
|
|
|
|
|
|
for (auto& s : values) {
|
|
|
|
KeyRef k = s.key.removePrefix(keyServersPrefix);
|
|
|
|
results.push_back_deep(results.arena(), k);
|
|
|
|
}
|
|
|
|
|
|
|
|
return results;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
|
|
|
Key begin,
|
2021-03-11 02:06:03 +08:00
|
|
|
Key end,
|
2017-05-26 04:48:44 +08:00
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2020-11-07 15:50:55 +08:00
|
|
|
auto task = makeReference<Task>(BackupRangeTaskFunc::name, BackupRangeTaskFunc::version, doneKey);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[BackupAgentBase::keyBeginKey] = begin;
|
|
|
|
task->params[BackupAgentBase::keyEndKey] = end;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state Reference<FlowLock> lock(new FlowLock(CLIENT_KNOBS->BACKUP_LOCK_BYTES));
|
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyConfig)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
wait(checkTaskVersion(cx, task, BackupRangeTaskFunc::name, BackupRangeTaskFunc::version));
|
|
|
|
// Find out if there is a shard boundary in(beginKey, endKey)
|
|
|
|
Standalone<VectorRef<KeyRef>> keys =
|
|
|
|
wait(runRYWTransaction(taskBucket->src, [=](Reference<ReadYourWritesTransaction> tr) {
|
|
|
|
return getBlockOfShards(tr,
|
|
|
|
task->params[DatabaseBackupAgent::keyBeginKey],
|
|
|
|
task->params[DatabaseBackupAgent::keyEndKey],
|
|
|
|
CLIENT_KNOBS->BACKUP_SHARD_TASK_LIMIT);
|
2021-03-11 02:06:03 +08:00
|
|
|
}));
|
2017-05-26 04:48:44 +08:00
|
|
|
if (keys.size() > 0) {
|
|
|
|
task->params[BackupRangeTaskFunc::keyAddBackupRangeTasks] = BinaryWriter::toValue(keys, IncludeVersion());
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Read everything from beginKey to endKey, write it to an output file, run the output file processor, and
|
|
|
|
// then set on_done.If we are still writing after X seconds, end the output file and insert a new backup_range
|
|
|
|
// task for the remainder.
|
|
|
|
state double timeout = now() + CLIENT_KNOBS->BACKUP_RANGE_TIMEOUT;
|
|
|
|
state Key addPrefix = task->params[DatabaseBackupAgent::keyAddPrefix];
|
|
|
|
state Key removePrefix = task->params[DatabaseBackupAgent::keyRemovePrefix];
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
state KeyRange range(
|
|
|
|
KeyRangeRef(task->params[BackupAgentBase::keyBeginKey], task->params[BackupAgentBase::keyEndKey]));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// retrieve kvData
|
|
|
|
state PromiseStream<RangeResultWithVersion> results;
|
|
|
|
|
2021-07-05 04:14:25 +08:00
|
|
|
state Future<Void> rc = readCommitted(
|
2021-07-17 15:11:40 +08:00
|
|
|
taskBucket->src, results, lock, range, Terminator::True, AccessSystemKeys::True, LockAware::True);
|
2017-05-26 04:48:44 +08:00
|
|
|
state Key rangeBegin = range.begin;
|
|
|
|
state Key rangeEnd;
|
|
|
|
state bool endOfStream = false;
|
|
|
|
state RangeResultWithVersion nextValues;
|
|
|
|
state int64_t nextValuesSize = 0;
|
|
|
|
nextValues.second = invalidVersion;
|
|
|
|
loop {
|
|
|
|
if (endOfStream && nextValues.second == invalidVersion) {
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
state RangeResultWithVersion values = std::move(nextValues);
|
|
|
|
state int64_t valuesSize = nextValuesSize;
|
|
|
|
nextValues = RangeResultWithVersion();
|
|
|
|
nextValues.second = invalidVersion;
|
|
|
|
nextValuesSize = 0;
|
|
|
|
|
|
|
|
if (!endOfStream) {
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
2017-05-26 04:48:44 +08:00
|
|
|
try {
|
|
|
|
RangeResultWithVersion v = waitNext(results.getFuture());
|
|
|
|
int64_t resultSize = v.first.expectedSize();
|
|
|
|
lock->release(resultSize);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (values.second == invalidVersion) {
|
|
|
|
values = v;
|
|
|
|
} else if ((values.second != v.second) ||
|
|
|
|
(valuesSize > 0 && resultSize > 0 &&
|
|
|
|
valuesSize + resultSize > CLIENT_KNOBS->BACKUP_LOG_WRITE_BATCH_MAX_SIZE)) {
|
|
|
|
nextValues = v;
|
|
|
|
nextValuesSize = resultSize;
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
|
|
|
} else {
|
2017-05-26 04:48:44 +08:00
|
|
|
values.first.append_deep(values.first.arena(), v.first.begin(), v.first.size());
|
|
|
|
values.first.more = v.first.more;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
valuesSize += resultSize;
|
|
|
|
} catch (Error& e) {
|
|
|
|
state Error err = e;
|
|
|
|
if (err.code() == error_code_actor_cancelled)
|
|
|
|
throw err;
|
|
|
|
|
|
|
|
if (err.code() == error_code_end_of_stream) {
|
|
|
|
endOfStream = true;
|
|
|
|
if (values.second != invalidVersion)
|
|
|
|
break;
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(logError(cx,
|
2017-05-26 04:48:44 +08:00
|
|
|
Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyErrors)
|
|
|
|
.pack(task->params[BackupAgentBase::keyConfigLogUid]),
|
|
|
|
format("ERROR: %s", err.what())));
|
|
|
|
|
|
|
|
throw err;
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if (now() >= timeout) {
|
|
|
|
task->params[BackupRangeTaskFunc::keyBackupRangeBeginKey] = rangeBegin;
|
2017-12-01 09:18:57 +08:00
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
rangeEnd = values.first.more ? keyAfter(values.first.end()[-1].key) : range.end;
|
|
|
|
|
|
|
|
state int valueLoc = 0;
|
|
|
|
state int committedValueLoc = 0;
|
|
|
|
state Reference<ReadYourWritesTransaction> tr = makeReference<ReadYourWritesTransaction>(cx);
|
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
try {
|
2017-12-02 23:02:26 +08:00
|
|
|
tr->reset();
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Key prefix = task->params[BackupAgentBase::keyConfigLogUid].withPrefix(
|
|
|
|
applyMutationsKeyVersionMapRange.begin);
|
|
|
|
state Key rangeCountKey = task->params[BackupAgentBase::keyConfigLogUid].withPrefix(
|
|
|
|
applyMutationsKeyVersionCountRange.begin);
|
2021-05-04 04:14:16 +08:00
|
|
|
state Future<RangeResult> backupVersions =
|
2017-05-26 04:48:44 +08:00
|
|
|
krmGetRanges(tr, prefix, KeyRangeRef(rangeBegin, rangeEnd), BUGGIFY ? 2 : 2000, 1e5);
|
2021-07-03 12:41:50 +08:00
|
|
|
state Future<Optional<Value>> logVersionValue =
|
|
|
|
tr->get(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsEndRange.begin),
|
2021-07-17 15:11:40 +08:00
|
|
|
Snapshot::True);
|
|
|
|
state Future<Optional<Value>> rangeCountValue = tr->get(rangeCountKey, Snapshot::True);
|
2021-07-03 12:41:50 +08:00
|
|
|
state Future<RangeResult> prevRange = tr->getRange(firstGreaterOrEqual(prefix),
|
|
|
|
lastLessOrEqual(rangeBegin.withPrefix(prefix)),
|
|
|
|
1,
|
2021-07-17 15:11:40 +08:00
|
|
|
Snapshot::True,
|
|
|
|
Reverse::True);
|
2021-05-04 04:14:16 +08:00
|
|
|
state Future<RangeResult> nextRange = tr->getRange(firstGreaterOrEqual(rangeEnd.withPrefix(prefix)),
|
|
|
|
firstGreaterOrEqual(strinc(prefix)),
|
|
|
|
1,
|
2021-07-17 15:11:40 +08:00
|
|
|
Snapshot::True,
|
|
|
|
Reverse::False);
|
2017-05-26 04:48:44 +08:00
|
|
|
state Future<Void> verified = taskBucket->keepRunning(tr, task);
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkDatabaseLock(tr,
|
|
|
|
BinaryReader::fromStringRef<UID>(
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[BackupAgentBase::keyConfigLogUid], Unversioned())));
|
|
|
|
wait(success(backupVersions) && success(logVersionValue) && success(rangeCountValue) &&
|
|
|
|
success(prevRange) && success(nextRange) && success(verified));
|
|
|
|
|
|
|
|
int64_t rangeCount = 0;
|
|
|
|
if (rangeCountValue.get().present()) {
|
|
|
|
ASSERT(rangeCountValue.get().get().size() == sizeof(int64_t));
|
|
|
|
memcpy(&rangeCount, rangeCountValue.get().get().begin(), rangeCountValue.get().get().size());
|
|
|
|
}
|
|
|
|
|
|
|
|
bool prevAdjacent =
|
|
|
|
prevRange.get().size() && prevRange.get()[0].value.size() &&
|
|
|
|
BinaryReader::fromStringRef<Version>(prevRange.get()[0].value, Unversioned()) != invalidVersion;
|
|
|
|
bool nextAdjacent =
|
|
|
|
nextRange.get().size() && nextRange.get()[0].value.size() &&
|
|
|
|
BinaryReader::fromStringRef<Version>(nextRange.get()[0].value, Unversioned()) != invalidVersion;
|
|
|
|
|
2019-03-19 06:03:43 +08:00
|
|
|
if ((!prevAdjacent || !nextAdjacent) &&
|
|
|
|
rangeCount > ((prevAdjacent || nextAdjacent) ? CLIENT_KNOBS->BACKUP_MAP_KEY_UPPER_LIMIT
|
|
|
|
: CLIENT_KNOBS->BACKUP_MAP_KEY_LOWER_LIMIT)) {
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "range insert delayed because too versionMap is too large");
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if (rangeCount > CLIENT_KNOBS->BACKUP_MAP_KEY_UPPER_LIMIT)
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent(SevWarnAlways, "DBA_KeyRangeMapTooLarge").log();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
wait(delay(1));
|
|
|
|
task->params[BackupRangeTaskFunc::keyBackupRangeBeginKey] = rangeBegin;
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
Version logVersion =
|
|
|
|
logVersionValue.get().present()
|
|
|
|
? BinaryReader::fromStringRef<Version>(logVersionValue.get().get(), Unversioned())
|
2021-07-03 12:41:50 +08:00
|
|
|
: ::invalidVersion;
|
2017-05-26 04:48:44 +08:00
|
|
|
if (logVersion >= values.second) {
|
|
|
|
task->params[BackupRangeTaskFunc::keyBackupRangeBeginKey] = rangeBegin;
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
//TraceEvent("DBA_Range").detail("Range", KeyRangeRef(rangeBegin, rangeEnd)).detail("Version", values.second).detail("Size", values.first.size()).detail("LogUID", task->params[BackupAgentBase::keyConfigLogUid]).detail("AddPrefix", addPrefix).detail("RemovePrefix", removePrefix);
|
|
|
|
|
|
|
|
Subspace krv(conf.get(DatabaseBackupAgent::keyRangeVersions));
|
|
|
|
state KeyRange versionRange = singleKeyRange(krv.pack(values.second));
|
|
|
|
tr->addReadConflictRange(versionRange);
|
|
|
|
tr->addWriteConflictRange(versionRange);
|
|
|
|
|
|
|
|
int versionLoc = 0;
|
|
|
|
std::vector<Future<Void>> setRanges;
|
|
|
|
state int64_t bytesSet = 0;
|
|
|
|
|
|
|
|
loop {
|
|
|
|
while (versionLoc < backupVersions.get().size() - 1 &&
|
|
|
|
(backupVersions.get()[versionLoc].value.size() < sizeof(Version) ||
|
|
|
|
BinaryReader::fromStringRef<Version>(backupVersions.get()[versionLoc].value,
|
|
|
|
Unversioned()) != invalidVersion)) {
|
|
|
|
versionLoc++;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if (versionLoc == backupVersions.get().size() - 1)
|
|
|
|
break;
|
|
|
|
|
|
|
|
if (backupVersions.get()[versionLoc + 1].key ==
|
2022-09-17 06:55:54 +08:00
|
|
|
(removePrefix == StringRef() ? allKeys.end : strinc(removePrefix))) {
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->clear(KeyRangeRef(
|
|
|
|
backupVersions.get()[versionLoc].key.removePrefix(removePrefix).withPrefix(addPrefix),
|
2022-09-17 06:55:54 +08:00
|
|
|
addPrefix == StringRef() ? allKeys.end : strinc(addPrefix)));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->clear(KeyRangeRef(backupVersions.get()[versionLoc].key,
|
|
|
|
backupVersions.get()[versionLoc + 1].key)
|
|
|
|
.removePrefix(removePrefix)
|
|
|
|
.withPrefix(addPrefix));
|
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
setRanges.push_back(krmSetRange(
|
2021-03-11 02:06:03 +08:00
|
|
|
tr,
|
|
|
|
prefix,
|
2018-08-11 04:57:10 +08:00
|
|
|
KeyRangeRef(backupVersions.get()[versionLoc].key, backupVersions.get()[versionLoc + 1].key),
|
|
|
|
BinaryWriter::toValue(values.second, Unversioned())));
|
2017-05-26 04:48:44 +08:00
|
|
|
int64_t added = 1;
|
|
|
|
tr->atomicOp(rangeCountKey, StringRef((uint8_t*)&added, 8), MutationRef::AddValue);
|
|
|
|
|
|
|
|
for (; valueLoc < values.first.size(); ++valueLoc) {
|
2019-03-19 06:03:43 +08:00
|
|
|
if (values.first[valueLoc].key >= backupVersions.get()[versionLoc + 1].key)
|
|
|
|
break;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if (values.first[valueLoc].key >= backupVersions.get()[versionLoc].key) {
|
|
|
|
//TraceEvent("DBA_Set", debugID).detail("Key", values.first[valueLoc].key).detail("Value", values.first[valueLoc].value);
|
|
|
|
tr->set(values.first[valueLoc].key.removePrefix(removePrefix).withPrefix(addPrefix),
|
|
|
|
values.first[valueLoc].value);
|
|
|
|
bytesSet += values.first[valueLoc].expectedSize() - removePrefix.expectedSize() +
|
|
|
|
addPrefix.expectedSize();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
versionLoc++;
|
|
|
|
}
|
|
|
|
|
|
|
|
wait(waitForAll(setRanges));
|
|
|
|
|
2019-09-28 09:32:27 +08:00
|
|
|
wait(tr->commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
Params.bytesWritten().set(task, Params.bytesWritten().getOrDefault(task) + bytesSet);
|
|
|
|
//TraceEvent("DBA_SetComplete", debugID).detail("Ver", values.second).detail("LogVersion", logVersion).detail("ReadVersion", readVer).detail("CommitVer", tr.getCommittedVersion()).detail("Range", versionRange);
|
|
|
|
|
|
|
|
if (backupVersions.get().more) {
|
|
|
|
tr->reset();
|
|
|
|
committedValueLoc = valueLoc;
|
|
|
|
rangeBegin = backupVersions.get().end()[-1].key;
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
|
|
|
break;
|
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
} catch (Error& e) {
|
2017-05-26 04:48:44 +08:00
|
|
|
wait(tr->onError(e));
|
|
|
|
valueLoc = committedValueLoc;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
rangeBegin = rangeEnd;
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> startBackupRangeInternal(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Standalone<VectorRef<KeyRef>> keys,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task,
|
|
|
|
Reference<TaskFuture> onDone) {
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Key nextKey = task->params[BackupAgentBase::keyBeginKey];
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
std::vector<Future<Key>> addTaskVector;
|
|
|
|
for (int idx = 0; idx < keys.size(); ++idx) {
|
2017-05-26 04:48:44 +08:00
|
|
|
if (nextKey != keys[idx]) {
|
2018-08-11 04:57:10 +08:00
|
|
|
addTaskVector.push_back(
|
|
|
|
addTask(tr, taskBucket, task, nextKey, keys[idx], TaskCompletionKey::joinWith(onDone)));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
nextKey = keys[idx];
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
if (nextKey != task->params[BackupAgentBase::keyEndKey]) {
|
2018-08-11 04:57:10 +08:00
|
|
|
addTaskVector.push_back(addTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
|
|
|
task,
|
|
|
|
nextKey,
|
|
|
|
task->params[BackupAgentBase::keyEndKey],
|
2018-08-11 04:57:10 +08:00
|
|
|
TaskCompletionKey::joinWith(onDone)));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
wait(waitForAll(addTaskVector));
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state Reference<TaskFuture> taskFuture = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
// Get the bytesWritten parameter from task and atomically add it to the rangeBytesWritten() property of the DR
|
2017-05-26 04:48:44 +08:00
|
|
|
// config.
|
|
|
|
DRConfig config(task);
|
2018-08-11 04:57:10 +08:00
|
|
|
int64_t bytesWritten = Params.bytesWritten().getOrDefault(task);
|
|
|
|
config.rangeBytesWritten().atomicOp(tr, bytesWritten, MutationRef::AddValue);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
if (task->params.find(BackupRangeTaskFunc::keyAddBackupRangeTasks) != task->params.end()) {
|
|
|
|
wait(startBackupRangeInternal(
|
2021-03-11 02:06:03 +08:00
|
|
|
tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
BinaryReader::fromStringRef<Standalone<VectorRef<KeyRef>>>(
|
|
|
|
task->params[BackupRangeTaskFunc::keyAddBackupRangeTasks], IncludeVersion()),
|
2018-06-07 04:05:53 +08:00
|
|
|
taskBucket,
|
2017-05-26 04:48:44 +08:00
|
|
|
futureBucket,
|
2018-08-11 04:57:10 +08:00
|
|
|
task,
|
|
|
|
taskFuture) &&
|
|
|
|
taskBucket->finish(tr, task));
|
|
|
|
} else if (task->params.find(BackupRangeTaskFunc::keyBackupRangeBeginKey) != task->params.end() &&
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[BackupRangeTaskFunc::keyBackupRangeBeginKey] <
|
|
|
|
task->params[BackupAgentBase::keyEndKey]) {
|
|
|
|
ASSERT(taskFuture->key.size() > 0);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(success(BackupRangeTaskFunc::addTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
task->params[BackupRangeTaskFunc::keyBackupRangeBeginKey],
|
|
|
|
task->params[BackupAgentBase::keyEndKey],
|
2017-05-26 04:48:44 +08:00
|
|
|
TaskCompletionKey::signal(taskFuture->key))) &&
|
2018-08-11 04:57:10 +08:00
|
|
|
taskBucket->finish(tr, task));
|
2017-05-26 04:48:44 +08:00
|
|
|
} else {
|
|
|
|
wait(taskFuture->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
|
|
|
}
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2021-01-26 10:03:09 +08:00
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef BackupRangeTaskFunc::name = "dr_backup_range"_sr;
|
|
|
|
const Key BackupRangeTaskFunc::keyAddBackupRangeTasks = "addBackupRangeTasks"_sr;
|
|
|
|
const Key BackupRangeTaskFunc::keyBackupRangeBeginKey = "backupRangeBeginKey"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(BackupRangeTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
struct FinishFullBackupTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
|
|
|
state Subspace states = Subspace(databaseBackupPrefixRange.begin)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(BackupAgentBase::keyStates)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(tr, task, FinishFullBackupTaskFunc::name, FinishFullBackupTaskFunc::version));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// Enable the stop key
|
|
|
|
Transaction srcTr(taskBucket->src);
|
|
|
|
srcTr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
Version readVersion = wait(srcTr.getReadVersion());
|
2021-01-26 10:03:09 +08:00
|
|
|
tr->set(states.pack(DatabaseBackupAgent::keyCopyStop), BinaryWriter::toValue(readVersion, Unversioned()));
|
|
|
|
TraceEvent("DBA_FinishFullBackup").detail("CopyStop", readVersion);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-06-07 04:05:53 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
2018-06-07 04:05:53 +08:00
|
|
|
Reference<Task> parentTask,
|
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
// After the BackupRangeTask completes, set the stop key which will stop the BackupLogsTask
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2020-11-07 15:50:55 +08:00
|
|
|
auto task = makeReference<Task>(FinishFullBackupTaskFunc::name, FinishFullBackupTaskFunc::version, doneKey);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
2020-07-03 16:36:20 +08:00
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
2020-07-03 16:36:20 +08:00
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef FinishFullBackupTaskFunc::name = "dr_finish_full_backup"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(FinishFullBackupTaskFunc);
|
|
|
|
|
|
|
|
struct EraseLogRangeTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _execute(cx, tb, fb, task);
|
|
|
|
};
|
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
|
|
|
};
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state FlowLock lock(CLIENT_KNOBS->BACKUP_LOCK_BYTES);
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(cx, task, EraseLogRangeTaskFunc::name, EraseLogRangeTaskFunc::version));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-09-28 09:32:27 +08:00
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(taskBucket->src));
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
Version endVersion = BinaryReader::fromStringRef<Version>(
|
|
|
|
task->params[DatabaseBackupAgent::keyEndVersion], Unversioned());
|
|
|
|
wait(eraseLogData(
|
|
|
|
tr,
|
|
|
|
task->params[BackupAgentBase::keyConfigLogUid],
|
|
|
|
task->params[BackupAgentBase::destUid],
|
|
|
|
Optional<Version>(endVersion),
|
2021-07-17 15:11:40 +08:00
|
|
|
CheckBackupUID::True,
|
2019-09-28 09:32:27 +08:00
|
|
|
BinaryReader::fromStringRef<Version>(task->params[BackupAgentBase::keyFolderId], Unversioned())));
|
|
|
|
wait(tr->commit());
|
|
|
|
return Void();
|
|
|
|
} catch (Error& e) {
|
|
|
|
wait(tr->onError(e));
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-06-07 04:05:53 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
|
|
|
Version endVersion,
|
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2020-11-07 15:50:55 +08:00
|
|
|
auto task = makeReference<Task>(EraseLogRangeTaskFunc::name, EraseLogRangeTaskFunc::version, doneKey, 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyBeginVersion] =
|
2018-06-07 04:34:22 +08:00
|
|
|
BinaryWriter::toValue(1, Unversioned()); // FIXME: remove in 6.X, only needed for 5.2 backward compatibility
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyEndVersion] = BinaryWriter::toValue(endVersion, Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2017-05-26 04:48:44 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2017-05-26 04:48:44 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2018-08-11 04:57:10 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<FutureBucket> futureBucket,
|
2020-11-07 15:50:55 +08:00
|
|
|
Reference<Task> task) {
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-06-07 04:34:22 +08:00
|
|
|
state Reference<TaskFuture> taskFuture = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
wait(taskFuture->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef EraseLogRangeTaskFunc::name = "dr_erase_log_range"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(EraseLogRangeTaskFunc);
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
struct CopyLogRangeTaskFunc : TaskFuncBase {
|
2017-05-26 04:48:44 +08:00
|
|
|
static StringRef name;
|
|
|
|
static constexpr uint32_t version = 1;
|
|
|
|
|
|
|
|
static struct {
|
2022-10-01 07:05:47 +08:00
|
|
|
static TaskParam<int64_t> bytesWritten() { return __FUNCTION__sr; }
|
2017-05-26 04:48:44 +08:00
|
|
|
} Params;
|
|
|
|
|
|
|
|
static const Key keyNextBeginVersion;
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
2018-08-11 04:57:10 +08:00
|
|
|
Reference<Task> task) override {
|
2021-01-26 10:03:09 +08:00
|
|
|
return _execute(cx, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2018-08-11 04:57:10 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
2018-08-11 04:57:10 +08:00
|
|
|
return _finish(tr, tb, fb, task);
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
|
|
|
|
|
|
|
// store mutation data from results until the end of stream or the timeout. If breaks on timeout returns the first
|
|
|
|
// uncopied version
|
2020-06-16 18:59:47 +08:00
|
|
|
ACTOR static Future<Optional<Version>> dumpData(Database cx,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<Task> task,
|
2020-07-03 16:36:20 +08:00
|
|
|
PromiseStream<RCGroup> results,
|
|
|
|
FlowLock* lock,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<TaskBucket> tb,
|
2020-06-16 18:59:47 +08:00
|
|
|
double breakTime) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state bool endOfStream = false;
|
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyConfig)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2021-05-04 04:14:16 +08:00
|
|
|
state std::vector<RangeResult> nextMutations;
|
2017-05-26 04:48:44 +08:00
|
|
|
state bool isTimeoutOccured = false;
|
2020-06-16 18:59:47 +08:00
|
|
|
state Optional<KeyRef> lastKey;
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version lastVersion;
|
|
|
|
state int64_t nextMutationSize = 0;
|
|
|
|
loop {
|
2021-03-11 02:06:03 +08:00
|
|
|
try {
|
2017-05-26 04:48:44 +08:00
|
|
|
if (endOfStream && !nextMutationSize) {
|
|
|
|
return Optional<Version>();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-05-04 04:14:16 +08:00
|
|
|
state std::vector<RangeResult> mutations = std::move(nextMutations);
|
2017-05-26 04:48:44 +08:00
|
|
|
state int64_t mutationSize = nextMutationSize;
|
2021-05-04 04:14:16 +08:00
|
|
|
nextMutations = std::vector<RangeResult>();
|
2017-05-26 04:48:44 +08:00
|
|
|
nextMutationSize = 0;
|
|
|
|
|
|
|
|
if (!endOfStream) {
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2017-05-26 04:48:44 +08:00
|
|
|
RCGroup group = waitNext(results.getFuture());
|
|
|
|
lock->release(group.items.expectedSize());
|
|
|
|
|
|
|
|
int vecSize = group.items.expectedSize();
|
|
|
|
if (mutationSize + vecSize >= CLIENT_KNOBS->BACKUP_LOG_WRITE_BATCH_MAX_SIZE) {
|
|
|
|
|
|
|
|
nextMutations.push_back(group.items);
|
|
|
|
nextMutationSize = vecSize;
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
mutations.push_back(group.items);
|
|
|
|
mutationSize += vecSize;
|
|
|
|
} catch (Error& e) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state Error error = e;
|
|
|
|
if (e.code() == error_code_end_of_stream) {
|
|
|
|
endOfStream = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
throw error;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-06-16 18:59:47 +08:00
|
|
|
state Optional<Version> nextVersionAfterBreak;
|
2017-05-26 04:48:44 +08:00
|
|
|
state Transaction tr(cx);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
2021-12-18 03:57:39 +08:00
|
|
|
tr.trState->options.sizeLimit = 2 * CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT;
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkDatabaseLock(&tr,
|
|
|
|
BinaryReader::fromStringRef<UID>(
|
|
|
|
task->params[BackupAgentBase::keyConfigLogUid], Unversioned())));
|
2017-05-26 04:48:44 +08:00
|
|
|
state int64_t bytesSet = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
bool first = true;
|
|
|
|
for (auto m : mutations) {
|
|
|
|
for (auto kv : m) {
|
2020-06-16 18:59:47 +08:00
|
|
|
if (isTimeoutOccured) {
|
2020-07-03 16:36:20 +08:00
|
|
|
Version newVersion = getLogKeyVersion(kv.key);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-07-03 16:36:20 +08:00
|
|
|
if (newVersion > lastVersion) {
|
|
|
|
nextVersionAfterBreak = newVersion;
|
|
|
|
break;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
if (first) {
|
|
|
|
tr.addReadConflictRange(singleKeyRange(kv.key));
|
|
|
|
first = false;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
tr.set(kv.key.removePrefix(backupLogKeys.begin)
|
|
|
|
.removePrefix(task->params[BackupAgentBase::destUid])
|
|
|
|
.withPrefix(task->params[BackupAgentBase::keyConfigLogUid])
|
|
|
|
.withPrefix(applyLogKeys.begin),
|
|
|
|
kv.value);
|
2017-05-26 04:48:44 +08:00
|
|
|
bytesSet += kv.expectedSize() - backupLogKeys.begin.expectedSize() +
|
|
|
|
applyLogKeys.begin.expectedSize();
|
2020-06-16 18:59:47 +08:00
|
|
|
lastKey = kv.key;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr.commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
Params.bytesWritten().set(task, Params.bytesWritten().getOrDefault(task) + bytesSet);
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2018-04-27 08:24:40 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr.onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
2020-06-16 18:59:47 +08:00
|
|
|
if (nextVersionAfterBreak.present()) {
|
2020-07-03 16:36:20 +08:00
|
|
|
return nextVersionAfterBreak;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!isTimeoutOccured && timer_monotonic() >= breakTime && lastKey.present()) {
|
2020-07-03 16:36:20 +08:00
|
|
|
// timeout occured
|
|
|
|
// continue to copy mutations with the
|
|
|
|
// same version before break because
|
|
|
|
// the next run should start from the beginning of a version > lastVersion.
|
|
|
|
lastVersion = getLogKeyVersion(lastKey.get());
|
|
|
|
isTimeoutOccured = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled || e.code() == error_code_backup_error)
|
|
|
|
throw e;
|
|
|
|
|
|
|
|
state Error err = e;
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(logError(cx,
|
|
|
|
Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyErrors)
|
|
|
|
.pack(task->params[BackupAgentBase::keyConfigLogUid]),
|
|
|
|
format("ERROR: Failed to dump mutations because of error %s", err.what())));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
throw err;
|
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
2020-06-16 18:59:47 +08:00
|
|
|
// state Reference<FlowLock> lock(new FlowLock(CLIENT_KNOBS->BACKUP_LOCK_BYTES));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(cx, task, CopyLogRangeTaskFunc::name, CopyLogRangeTaskFunc::version));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
state Version beginVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyBeginVersion], Unversioned());
|
|
|
|
state Version endVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyEndVersion], Unversioned());
|
2020-06-16 18:59:47 +08:00
|
|
|
|
|
|
|
Version newEndVersion = std::min(endVersion,
|
|
|
|
(((beginVersion - 1) / CLIENT_KNOBS->COPY_LOG_BLOCK_SIZE) + 1 +
|
|
|
|
CLIENT_KNOBS->COPY_LOG_BLOCKS_PER_TASK +
|
|
|
|
(g_network->isSimulated() ? CLIENT_KNOBS->BACKUP_SIM_COPY_LOG_RANGES : 0)) *
|
|
|
|
CLIENT_KNOBS->COPY_LOG_BLOCK_SIZE);
|
|
|
|
|
|
|
|
state Standalone<VectorRef<KeyRangeRef>> ranges = getLogRanges(
|
|
|
|
beginVersion, newEndVersion, task->params[BackupAgentBase::destUid], CLIENT_KNOBS->COPY_LOG_BLOCK_SIZE);
|
|
|
|
state int nRanges = ranges.size();
|
2020-07-03 16:36:20 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state std::vector<PromiseStream<RCGroup>> results;
|
|
|
|
state std::vector<Future<Void>> rc;
|
2020-06-16 18:59:47 +08:00
|
|
|
state std::vector<Reference<FlowLock>> locks;
|
2020-07-03 16:36:20 +08:00
|
|
|
state Version nextVersion = beginVersion;
|
|
|
|
state double breakTime = timer_monotonic() + CLIENT_KNOBS->COPY_LOG_TASK_DURATION_NANOS;
|
|
|
|
state int rangeN = 0;
|
|
|
|
|
|
|
|
loop {
|
|
|
|
if (rangeN >= nRanges)
|
|
|
|
break;
|
|
|
|
|
|
|
|
// prefetch
|
2020-11-07 15:50:55 +08:00
|
|
|
int prefetchTo = std::min(rangeN + CLIENT_KNOBS->COPY_LOG_PREFETCH_BLOCKS, nRanges);
|
2020-07-03 16:36:20 +08:00
|
|
|
|
|
|
|
for (int j = results.size(); j < prefetchTo; j++) {
|
|
|
|
results.push_back(PromiseStream<RCGroup>());
|
|
|
|
locks.push_back(makeReference<FlowLock>(CLIENT_KNOBS->COPY_LOG_READ_AHEAD_BYTES));
|
|
|
|
rc.push_back(readCommitted(taskBucket->src,
|
|
|
|
results[j],
|
|
|
|
Future<Void>(Void()),
|
|
|
|
locks[j],
|
|
|
|
ranges[j],
|
|
|
|
decodeBKMutationLogKey,
|
2021-07-17 15:11:40 +08:00
|
|
|
Terminator::True,
|
|
|
|
AccessSystemKeys::True,
|
|
|
|
LockAware::True));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2020-07-03 16:36:20 +08:00
|
|
|
// copy the range
|
|
|
|
Optional<Version> nextVersionBr =
|
|
|
|
wait(dumpData(cx, task, results[rangeN], locks[rangeN].getPtr(), taskBucket, breakTime));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-06-16 18:59:47 +08:00
|
|
|
// exit from the task if a timeout occurs
|
2020-07-03 16:36:20 +08:00
|
|
|
if (nextVersionBr.present()) {
|
|
|
|
nextVersion = nextVersionBr.get();
|
|
|
|
// cancel prefetch
|
2020-07-05 02:37:28 +08:00
|
|
|
TraceEvent(SevInfo, "CopyLogRangeTaskFuncAborted")
|
2020-07-03 16:36:20 +08:00
|
|
|
.detail("DurationNanos", CLIENT_KNOBS->COPY_LOG_TASK_DURATION_NANOS)
|
|
|
|
.detail("RangeN", rangeN)
|
|
|
|
.detail("BytesWritten", Params.bytesWritten().getOrDefault(task));
|
|
|
|
for (int j = results.size(); --j >= rangeN;)
|
|
|
|
rc[j].cancel();
|
|
|
|
break;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
// the whole range has been dumped
|
|
|
|
nextVersion = getLogKeyVersion(ranges[rangeN].end);
|
2021-03-11 02:06:03 +08:00
|
|
|
rangeN++;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
if (nextVersion < endVersion) {
|
|
|
|
task->params[CopyLogRangeTaskFunc::keyNextBeginVersion] = BinaryWriter::toValue(nextVersion, Unversioned());
|
|
|
|
}
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
|
|
|
Version beginVersion,
|
2018-04-27 08:24:40 +08:00
|
|
|
Version endVersion,
|
2017-05-26 04:48:44 +08:00
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
|
|
|
auto task = makeReference<Task>(CopyLogRangeTaskFunc::name, CopyLogRangeTaskFunc::version, doneKey, 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyBeginVersion] = BinaryWriter::toValue(beginVersion, Unversioned());
|
|
|
|
task->params[DatabaseBackupAgent::keyEndVersion] = BinaryWriter::toValue(endVersion, Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
|
|
|
task,
|
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
|
|
|
taskBucket,
|
|
|
|
task,
|
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version endVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyEndVersion], Unversioned());
|
2018-04-27 08:24:40 +08:00
|
|
|
state Reference<TaskFuture> taskFuture = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
// Get the bytesWritten parameter from task and atomically add it to the logBytesWritten() property of the DR
|
|
|
|
// config.
|
2017-05-26 04:48:44 +08:00
|
|
|
DRConfig config(task);
|
|
|
|
int64_t bytesWritten = Params.bytesWritten().getOrDefault(task);
|
|
|
|
config.logBytesWritten().atomicOp(tr, bytesWritten, MutationRef::AddValue);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (task->params.find(CopyLogRangeTaskFunc::keyNextBeginVersion) != task->params.end()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version nextVersion = BinaryReader::fromStringRef<Version>(
|
|
|
|
task->params[CopyLogRangeTaskFunc::keyNextBeginVersion], Unversioned());
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(success(CopyLogRangeTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, nextVersion, endVersion, TaskCompletionKey::signal(taskFuture->key))) &&
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket->finish(tr, task));
|
|
|
|
} else {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskFuture->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef CopyLogRangeTaskFunc::name = "dr_copy_log_range"_sr;
|
|
|
|
const Key CopyLogRangeTaskFunc::keyNextBeginVersion = "nextBeginVersion"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(CopyLogRangeTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
struct CopyLogsTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyConfig)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
|
|
|
state Subspace states = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyStates)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(tr, task, CopyLogsTaskFunc::name, CopyLogsTaskFunc::version));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
state Version beginVersion =
|
2017-05-26 04:48:44 +08:00
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyBeginVersion], Unversioned());
|
2018-02-21 05:22:31 +08:00
|
|
|
state Version prevBeginVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyPrevBeginVersion], Unversioned());
|
2017-05-26 04:48:44 +08:00
|
|
|
state Future<Optional<Value>> fStopValue = tr->get(states.pack(DatabaseBackupAgent::keyCopyStop));
|
|
|
|
state Future<Optional<Value>> fAppliedValue =
|
|
|
|
tr->get(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsBeginRange.begin));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
Transaction srcTr(taskBucket->src);
|
|
|
|
srcTr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Version endVersion = wait(srcTr.getReadVersion());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Reference<TaskFuture> onDone = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (endVersion <= beginVersion) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(CopyLogsTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, prevBeginVersion, beginVersion, TaskCompletionKey::signal(onDone))));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
Optional<Value> appliedValue = wait(fAppliedValue);
|
|
|
|
state Version appliedVersion =
|
|
|
|
appliedValue.present() ? BinaryReader::fromStringRef<Version>(appliedValue.get(), Unversioned()) : 100;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
state Version applyVersion =
|
|
|
|
std::max<Version>(appliedVersion, beginVersion - CLIENT_KNOBS->BACKUP_VERSION_DELAY);
|
2018-02-21 05:22:31 +08:00
|
|
|
Subspace krv = conf.get(DatabaseBackupAgent::keyRangeVersions);
|
|
|
|
KeyRange versionRange = KeyRangeRef(krv.pack(0), krv.pack(applyVersion + 1));
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->addReadConflictRange(versionRange);
|
|
|
|
tr->addWriteConflictRange(versionRange);
|
|
|
|
tr->set(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsEndRange.begin),
|
|
|
|
BinaryWriter::toValue(applyVersion, Unversioned()));
|
|
|
|
|
|
|
|
Optional<Value> stopValue = wait(fStopValue);
|
|
|
|
state Version stopVersionData =
|
|
|
|
stopValue.present() ? BinaryReader::fromStringRef<Version>(stopValue.get(), Unversioned()) : -1;
|
|
|
|
|
|
|
|
if (endVersion - beginVersion > deterministicRandom()->randomInt64(0, CLIENT_KNOBS->BACKUP_VERSION_DELAY)) {
|
|
|
|
TraceEvent("DBA_CopyLogs")
|
2019-03-19 06:03:43 +08:00
|
|
|
.detail("BeginVersion", beginVersion)
|
|
|
|
.detail("ApplyVersion", applyVersion)
|
|
|
|
.detail("EndVersion", endVersion)
|
2017-05-26 04:48:44 +08:00
|
|
|
.detail("StopVersionData", stopVersionData)
|
|
|
|
.detail("LogUID", task->params[BackupAgentBase::keyConfigLogUid]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if ((stopVersionData == -1) || (stopVersionData >= applyVersion)) {
|
|
|
|
state Reference<TaskFuture> allPartsDone = futureBucket->future(tr);
|
|
|
|
std::vector<Future<Key>> addTaskVector;
|
2018-02-21 05:22:31 +08:00
|
|
|
addTaskVector.push_back(CopyLogsTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, beginVersion, endVersion, TaskCompletionKey::signal(onDone), allPartsDone));
|
2017-05-26 04:48:44 +08:00
|
|
|
int blockSize = std::max<int>(
|
|
|
|
1, ((endVersion - beginVersion) / CLIENT_KNOBS->BACKUP_COPY_TASKS) / CLIENT_KNOBS->BACKUP_BLOCK_SIZE);
|
|
|
|
for (int64_t vblock = beginVersion / CLIENT_KNOBS->BACKUP_BLOCK_SIZE;
|
|
|
|
vblock < (endVersion + CLIENT_KNOBS->BACKUP_BLOCK_SIZE - 1) / CLIENT_KNOBS->BACKUP_BLOCK_SIZE;
|
|
|
|
vblock += blockSize) {
|
|
|
|
addTaskVector.push_back(CopyLogRangeTaskFunc::addTask(
|
2021-03-11 02:06:03 +08:00
|
|
|
tr,
|
2020-10-03 08:44:31 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2017-05-26 04:48:44 +08:00
|
|
|
std::max(beginVersion, vblock * CLIENT_KNOBS->BACKUP_BLOCK_SIZE),
|
|
|
|
std::min(endVersion, (vblock + blockSize) * CLIENT_KNOBS->BACKUP_BLOCK_SIZE),
|
2018-06-07 04:05:53 +08:00
|
|
|
TaskCompletionKey::joinWith(allPartsDone)));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
// Do not erase at the first time
|
|
|
|
if (prevBeginVersion > 0) {
|
2018-06-07 04:05:53 +08:00
|
|
|
addTaskVector.push_back(EraseLogRangeTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, beginVersion, TaskCompletionKey::joinWith(allPartsDone)));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitForAll(addTaskVector) && taskBucket->finish(tr, task));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2021-04-01 06:19:50 +08:00
|
|
|
if (appliedVersion < applyVersion) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(CopyLogsTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, prevBeginVersion, beginVersion, TaskCompletionKey::signal(onDone))));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
wait(onDone->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->set(states.pack(DatabaseBackupAgent::keyStateStop), BinaryWriter::toValue(beginVersion, Unversioned()));
|
|
|
|
}
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
2018-02-21 05:22:31 +08:00
|
|
|
Version prevBeginVersion,
|
|
|
|
Version beginVersion,
|
|
|
|
TaskCompletionKey completionKey,
|
2018-08-11 04:57:10 +08:00
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2020-11-07 15:50:55 +08:00
|
|
|
auto task = makeReference<Task>(CopyLogsTaskFunc::name, CopyLogsTaskFunc::version, doneKey, 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
|
|
|
task->params[BackupAgentBase::keyBeginVersion] = BinaryWriter::toValue(beginVersion, Unversioned());
|
|
|
|
task->params[DatabaseBackupAgent::keyPrevBeginVersion] = BinaryWriter::toValue(prevBeginVersion, Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
if (!waitFor) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2020-10-03 08:44:31 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef CopyLogsTaskFunc::name = "dr_copy_logs"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(CopyLogsTaskFunc);
|
|
|
|
|
|
|
|
struct FinishedFullBackupTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
|
|
|
static const Key keyInsertTask;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state Subspace sourceStates = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keySourceStates)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(cx, task, FinishedFullBackupTaskFunc::name, FinishedFullBackupTaskFunc::version));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
state Transaction tr2(cx);
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
tr2.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
Optional<Value> beginValue = wait(
|
|
|
|
tr2.get(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsBeginRange.begin)));
|
|
|
|
state Version appliedVersion =
|
|
|
|
beginValue.present() ? BinaryReader::fromStringRef<Version>(beginValue.get(), Unversioned()) : -1;
|
2018-02-21 05:22:31 +08:00
|
|
|
Optional<Value> endValue = wait(
|
2019-09-28 09:32:27 +08:00
|
|
|
tr2.get(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsEndRange.begin)));
|
|
|
|
Version endVersion =
|
2017-05-26 04:48:44 +08:00
|
|
|
endValue.present() ? BinaryReader::fromStringRef<Version>(endValue.get(), Unversioned()) : -1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
//TraceEvent("DBA_FinishedFullBackup").detail("Applied", appliedVersion).detail("EndVer", endVersion);
|
|
|
|
if (appliedVersion < endVersion) {
|
|
|
|
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
|
|
|
task->params[FinishedFullBackupTaskFunc::keyInsertTask] = StringRef();
|
2019-09-28 09:32:27 +08:00
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2018-04-27 08:24:40 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr2.onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(taskBucket->src));
|
|
|
|
state Key logUidValue = task->params[DatabaseBackupAgent::keyConfigLogUid];
|
|
|
|
state Key destUidValue = task->params[BackupAgentBase::destUid];
|
2020-11-07 15:50:55 +08:00
|
|
|
state Version backupUid =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[BackupAgentBase::keyFolderId], Unversioned());
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
2018-08-11 04:57:10 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
Optional<Value> v = wait(tr->get(sourceStates.pack(DatabaseBackupAgent::keyFolderId)));
|
2017-05-26 04:48:44 +08:00
|
|
|
if (v.present() && BinaryReader::fromStringRef<Version>(v.get(), Unversioned()) >
|
|
|
|
BinaryReader::fromStringRef<Version>(
|
|
|
|
task->params[DatabaseBackupAgent::keyFolderId], Unversioned()))
|
|
|
|
return Void();
|
|
|
|
|
2021-07-17 15:11:40 +08:00
|
|
|
wait(eraseLogData(tr, logUidValue, destUidValue, Optional<Version>(), CheckBackupUID::True, backupUid));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2018-04-27 08:24:40 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2018-04-27 08:24:40 +08:00
|
|
|
auto task = makeReference<Task>(FinishedFullBackupTaskFunc::name, FinishedFullBackupTaskFunc::version, doneKey);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
|
|
|
task,
|
2017-05-26 04:48:44 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
|
|
|
taskBucket,
|
|
|
|
task,
|
2017-05-26 04:48:44 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2018-02-21 05:22:31 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(BackupAgentBase::keyConfig)
|
2018-08-11 04:57:10 +08:00
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2018-04-27 08:24:40 +08:00
|
|
|
state Subspace states = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyStates)
|
2018-08-11 04:57:10 +08:00
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (task->params.find(FinishedFullBackupTaskFunc::keyInsertTask) != task->params.end()) {
|
|
|
|
state Reference<TaskFuture> onDone = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(FinishedFullBackupTaskFunc::addTask(tr, taskBucket, task, TaskCompletionKey::signal(onDone))));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
|
2020-11-07 15:50:55 +08:00
|
|
|
UID logUid =
|
2018-02-21 05:22:31 +08:00
|
|
|
BinaryReader::fromStringRef<UID>(task->params[DatabaseBackupAgent::keyConfigLogUid], Unversioned());
|
|
|
|
Key logsPath = uidPrefixKey(applyLogKeys.begin, logUid);
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->clear(KeyRangeRef(logsPath, strinc(logsPath)));
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
tr->clear(conf.range());
|
|
|
|
tr->set(states.pack(DatabaseBackupAgent::keyStateStatus),
|
2017-05-26 04:48:44 +08:00
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_COMPLETED)));
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
|
|
|
return Void();
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _execute(cx, tb, fb, task);
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
2020-10-03 08:44:31 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2018-04-27 08:24:40 +08:00
|
|
|
};
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef FinishedFullBackupTaskFunc::name = "dr_finished_full_backup"_sr;
|
|
|
|
const Key FinishedFullBackupTaskFunc::keyInsertTask = "insertTask"_sr;
|
2021-01-26 10:03:09 +08:00
|
|
|
REGISTER_TASKFUNC(FinishedFullBackupTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
struct CopyDiffLogsTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyConfig)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
|
|
|
state Subspace states = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyStates)
|
2018-08-11 04:57:10 +08:00
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
|
|
|
wait(checkTaskVersion(tr, task, CopyDiffLogsTaskFunc::name, CopyDiffLogsTaskFunc::version));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
state Version beginVersion =
|
2018-08-11 04:57:10 +08:00
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyBeginVersion], Unversioned());
|
2018-04-27 08:24:40 +08:00
|
|
|
state Version prevBeginVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyPrevBeginVersion], Unversioned());
|
|
|
|
state Future<Optional<Value>> fStopWhenDone = tr->get(conf.pack(DatabaseBackupAgent::keyConfigStopWhenDoneKey));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
Transaction srcTr(taskBucket->src);
|
|
|
|
srcTr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Version endVersion = wait(srcTr.getReadVersion());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
state Reference<TaskFuture> onDone = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (endVersion <= beginVersion) {
|
|
|
|
wait(delay(FLOW_KNOBS->PREVENT_FAST_SPIN_DELAY));
|
|
|
|
wait(success(CopyDiffLogsTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, prevBeginVersion, beginVersion, TaskCompletionKey::signal(onDone))));
|
|
|
|
wait(taskBucket->finish(tr, task));
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
tr->set(task->params[BackupAgentBase::keyConfigLogUid].withPrefix(applyMutationsEndRange.begin),
|
|
|
|
BinaryWriter::toValue(beginVersion, Unversioned()));
|
|
|
|
Optional<Value> stopWhenDone = wait(fStopWhenDone);
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
if (endVersion - beginVersion > deterministicRandom()->randomInt64(0, CLIENT_KNOBS->BACKUP_VERSION_DELAY)) {
|
|
|
|
TraceEvent("DBA_CopyDiffLogs")
|
|
|
|
.detail("BeginVersion", beginVersion)
|
|
|
|
.detail("EndVersion", endVersion)
|
|
|
|
.detail("LogUID", task->params[BackupAgentBase::keyConfigLogUid]);
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
|
|
|
|
2021-07-09 23:13:02 +08:00
|
|
|
// set the log version to the state
|
|
|
|
tr->set(StringRef(states.pack(DatabaseBackupAgent::keyStateLogBeginVersion)),
|
|
|
|
BinaryWriter::toValue(beginVersion, Unversioned()));
|
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (!stopWhenDone.present()) {
|
|
|
|
state Reference<TaskFuture> allPartsDone = futureBucket->future(tr);
|
|
|
|
std::vector<Future<Key>> addTaskVector;
|
|
|
|
addTaskVector.push_back(CopyDiffLogsTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, beginVersion, endVersion, TaskCompletionKey::signal(onDone), allPartsDone));
|
|
|
|
int blockSize = std::max<int>(
|
2017-05-26 04:48:44 +08:00
|
|
|
1, ((endVersion - beginVersion) / CLIENT_KNOBS->BACKUP_COPY_TASKS) / CLIENT_KNOBS->BACKUP_BLOCK_SIZE);
|
2018-04-27 08:24:40 +08:00
|
|
|
for (int64_t vblock = beginVersion / CLIENT_KNOBS->BACKUP_BLOCK_SIZE;
|
2017-05-26 04:48:44 +08:00
|
|
|
vblock < (endVersion + CLIENT_KNOBS->BACKUP_BLOCK_SIZE - 1) / CLIENT_KNOBS->BACKUP_BLOCK_SIZE;
|
|
|
|
vblock += blockSize) {
|
2018-04-27 08:24:40 +08:00
|
|
|
addTaskVector.push_back(CopyLogRangeTaskFunc::addTask(
|
2021-03-11 02:06:03 +08:00
|
|
|
tr,
|
2018-04-27 08:24:40 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2017-05-26 04:48:44 +08:00
|
|
|
std::max(beginVersion, vblock * CLIENT_KNOBS->BACKUP_BLOCK_SIZE),
|
|
|
|
std::min(endVersion, (vblock + blockSize) * CLIENT_KNOBS->BACKUP_BLOCK_SIZE),
|
2018-04-27 08:24:40 +08:00
|
|
|
TaskCompletionKey::joinWith(allPartsDone)));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (prevBeginVersion > 0) {
|
|
|
|
addTaskVector.push_back(EraseLogRangeTaskFunc::addTask(
|
2018-06-07 04:05:53 +08:00
|
|
|
tr, taskBucket, task, beginVersion, TaskCompletionKey::joinWith(allPartsDone)));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitForAll(addTaskVector) && taskBucket->finish(tr, task));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2018-04-27 08:24:40 +08:00
|
|
|
wait(onDone->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
2020-10-03 08:44:31 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
2018-04-27 08:24:40 +08:00
|
|
|
Reference<Task> parentTask,
|
|
|
|
Version prevBeginVersion,
|
|
|
|
Version beginVersion,
|
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2020-11-07 15:50:55 +08:00
|
|
|
auto task = makeReference<Task>(CopyDiffLogsTaskFunc::name, CopyDiffLogsTaskFunc::version, doneKey, 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyBeginVersion] = BinaryWriter::toValue(beginVersion, Unversioned());
|
|
|
|
task->params[DatabaseBackupAgent::keyPrevBeginVersion] = BinaryWriter::toValue(prevBeginVersion, Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
|
|
|
task,
|
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2018-04-27 08:24:40 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-04-27 08:24:40 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2018-08-11 04:57:10 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
|
|
|
StringRef getName() const override { return name; };
|
|
|
|
|
|
|
|
Future<Void> execute(Database cx,
|
2018-08-11 04:57:10 +08:00
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
2018-04-27 08:24:40 +08:00
|
|
|
return Void();
|
|
|
|
};
|
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
2018-04-27 08:24:40 +08:00
|
|
|
Reference<Task> task) override {
|
2021-01-26 10:03:09 +08:00
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef CopyDiffLogsTaskFunc::name = "dr_copy_diff_logs"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
REGISTER_TASKFUNC(CopyDiffLogsTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
// Skip unneeded EraseLogRangeTaskFunc in 5.1
|
|
|
|
struct SkipOldEraseLogRangeTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
state Reference<TaskFuture> taskFuture = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskFuture->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
2018-04-27 08:24:40 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
2018-04-27 08:24:40 +08:00
|
|
|
Reference<Task> task) override {
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2018-04-27 08:24:40 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef SkipOldEraseLogRangeTaskFunc::name = "dr_skip_legacy_task"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
REGISTER_TASKFUNC(SkipOldEraseLogRangeTaskFunc);
|
|
|
|
REGISTER_TASKFUNC_ALIAS(SkipOldEraseLogRangeTaskFunc, db_erase_log_range);
|
|
|
|
|
2018-06-09 02:11:08 +08:00
|
|
|
// This is almost the same as CopyLogRangeTaskFunc in 5.1. The only purpose is to support DR upgrade
|
2018-08-11 04:57:10 +08:00
|
|
|
struct OldCopyLogRangeTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
|
|
|
static constexpr uint32_t version = 1;
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
static struct {
|
2022-10-01 07:05:47 +08:00
|
|
|
static TaskParam<int64_t> bytesWritten() { return __FUNCTION__sr; }
|
2018-04-27 08:24:40 +08:00
|
|
|
} Params;
|
|
|
|
|
|
|
|
static const Key keyNextBeginVersion;
|
|
|
|
|
|
|
|
StringRef getName() const override { return name; };
|
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
2020-11-07 15:50:55 +08:00
|
|
|
Reference<Task> task) override {
|
|
|
|
return _execute(cx, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2020-11-07 15:50:55 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2018-04-27 08:24:40 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> dumpData(Database cx,
|
|
|
|
Reference<Task> task,
|
|
|
|
PromiseStream<RCGroup> results,
|
2020-07-03 16:36:20 +08:00
|
|
|
FlowLock* lock,
|
2018-04-27 08:24:40 +08:00
|
|
|
Reference<TaskBucket> tb) {
|
|
|
|
state bool endOfStream = false;
|
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyConfig)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2021-05-04 04:14:16 +08:00
|
|
|
state std::vector<RangeResult> nextMutations;
|
2017-05-26 04:48:44 +08:00
|
|
|
state int64_t nextMutationSize = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
2018-04-27 08:24:40 +08:00
|
|
|
try {
|
|
|
|
if (endOfStream && !nextMutationSize) {
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2021-05-04 04:14:16 +08:00
|
|
|
state std::vector<RangeResult> mutations = std::move(nextMutations);
|
2018-08-11 04:57:10 +08:00
|
|
|
state int64_t mutationSize = nextMutationSize;
|
2021-05-04 04:14:16 +08:00
|
|
|
nextMutations = std::vector<RangeResult>();
|
2018-04-27 08:24:40 +08:00
|
|
|
nextMutationSize = 0;
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
if (!endOfStream) {
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2021-01-26 10:03:09 +08:00
|
|
|
RCGroup group = waitNext(results.getFuture());
|
|
|
|
lock->release(group.items.expectedSize());
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
int vecSize = group.items.expectedSize();
|
2018-04-27 08:24:40 +08:00
|
|
|
if (mutationSize + vecSize >= CLIENT_KNOBS->BACKUP_LOG_WRITE_BATCH_MAX_SIZE) {
|
|
|
|
|
|
|
|
nextMutations.push_back(group.items);
|
|
|
|
nextMutationSize = vecSize;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
mutations.push_back(group.items);
|
|
|
|
mutationSize += vecSize;
|
|
|
|
} catch (Error& e) {
|
|
|
|
state Error error = e;
|
2018-08-11 04:57:10 +08:00
|
|
|
if (e.code() == error_code_end_of_stream) {
|
2018-04-27 08:24:40 +08:00
|
|
|
endOfStream = true;
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
throw error;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
state Transaction tr(cx);
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2018-04-27 08:24:40 +08:00
|
|
|
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
2021-12-18 03:57:39 +08:00
|
|
|
tr.trState->options.sizeLimit = 2 * CLIENT_KNOBS->TRANSACTION_SIZE_LIMIT;
|
2018-04-27 08:24:40 +08:00
|
|
|
wait(checkDatabaseLock(&tr,
|
|
|
|
BinaryReader::fromStringRef<UID>(
|
|
|
|
task->params[BackupAgentBase::keyConfigLogUid], Unversioned())));
|
|
|
|
state int64_t bytesSet = 0;
|
|
|
|
|
2019-10-02 09:52:07 +08:00
|
|
|
bool first = true;
|
|
|
|
for (auto m : mutations) {
|
2018-04-27 08:24:40 +08:00
|
|
|
for (auto kv : m) {
|
2019-10-02 09:52:07 +08:00
|
|
|
if (first) {
|
|
|
|
tr.addReadConflictRange(singleKeyRange(kv.key));
|
|
|
|
first = false;
|
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
tr.set(kv.key.removePrefix(backupLogKeys.begin).withPrefix(applyLogKeys.begin),
|
|
|
|
kv.value);
|
|
|
|
bytesSet += kv.expectedSize() - backupLogKeys.begin.expectedSize() +
|
|
|
|
applyLogKeys.begin.expectedSize();
|
|
|
|
}
|
|
|
|
}
|
2019-10-02 09:52:07 +08:00
|
|
|
|
|
|
|
wait(tr.commit());
|
|
|
|
Params.bytesWritten().set(task, Params.bytesWritten().getOrDefault(task) + bytesSet);
|
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
|
|
|
wait(tr.onError(e));
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() == error_code_actor_cancelled || e.code() == error_code_backup_error)
|
2021-03-11 02:06:03 +08:00
|
|
|
throw e;
|
2018-04-27 08:24:40 +08:00
|
|
|
|
|
|
|
state Error err = e;
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(logError(cx,
|
2018-02-21 05:22:31 +08:00
|
|
|
Subspace(databaseBackupPrefixRange.begin)
|
2018-08-11 04:57:10 +08:00
|
|
|
.get(BackupAgentBase::keyErrors)
|
2018-04-27 08:24:40 +08:00
|
|
|
.pack(task->params[BackupAgentBase::keyConfigLogUid]),
|
|
|
|
format("ERROR: Failed to dump mutations because of error %s", err.what())));
|
|
|
|
|
|
|
|
throw err;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
2018-04-27 08:24:40 +08:00
|
|
|
state Reference<FlowLock> lock(new FlowLock(CLIENT_KNOBS->BACKUP_LOCK_BYTES));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(cx, task, OldCopyLogRangeTaskFunc::name, OldCopyLogRangeTaskFunc::version));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version beginVersion =
|
2018-04-27 08:24:40 +08:00
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyBeginVersion], Unversioned());
|
|
|
|
state Version endVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyEndVersion], Unversioned());
|
|
|
|
state Version newEndVersion =
|
2017-05-26 04:48:44 +08:00
|
|
|
std::min(endVersion,
|
|
|
|
(((beginVersion - 1) / CLIENT_KNOBS->BACKUP_BLOCK_SIZE) + 2 +
|
2018-04-27 08:24:40 +08:00
|
|
|
(g_network->isSimulated() ? CLIENT_KNOBS->BACKUP_SIM_COPY_LOG_RANGES : 0)) *
|
|
|
|
CLIENT_KNOBS->BACKUP_BLOCK_SIZE);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
state Standalone<VectorRef<KeyRangeRef>> ranges = getLogRanges(beginVersion,
|
|
|
|
newEndVersion,
|
2019-03-19 06:03:43 +08:00
|
|
|
task->params[BackupAgentBase::keyConfigLogUid],
|
2018-04-27 08:24:40 +08:00
|
|
|
CLIENT_KNOBS->BACKUP_BLOCK_SIZE);
|
|
|
|
state std::vector<PromiseStream<RCGroup>> results;
|
|
|
|
state std::vector<Future<Void>> rc;
|
|
|
|
state std::vector<Future<Void>> dump;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
for (int i = 0; i < ranges.size(); ++i) {
|
|
|
|
results.push_back(PromiseStream<RCGroup>());
|
2020-11-07 15:50:55 +08:00
|
|
|
rc.push_back(readCommitted(taskBucket->src,
|
2018-04-27 08:24:40 +08:00
|
|
|
results[i],
|
2020-11-07 15:50:55 +08:00
|
|
|
Future<Void>(Void()),
|
2021-03-11 02:06:03 +08:00
|
|
|
lock,
|
2018-04-27 08:24:40 +08:00
|
|
|
ranges[i],
|
2020-11-07 15:50:55 +08:00
|
|
|
decodeBKMutationLogKey,
|
2021-07-17 15:11:40 +08:00
|
|
|
Terminator::True,
|
|
|
|
AccessSystemKeys::True,
|
|
|
|
LockAware::True));
|
2018-04-27 08:24:40 +08:00
|
|
|
dump.push_back(dumpData(cx, task, results[i], lock.getPtr(), taskBucket));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitForAll(dump));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (newEndVersion < endVersion) {
|
|
|
|
task->params[OldCopyLogRangeTaskFunc::keyNextBeginVersion] =
|
2018-02-21 05:22:31 +08:00
|
|
|
BinaryWriter::toValue(newEndVersion, Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2018-06-09 04:57:00 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
2019-02-13 08:07:17 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
2018-04-27 08:24:40 +08:00
|
|
|
Version beginVersion,
|
2019-02-13 08:07:17 +08:00
|
|
|
Version endVersion,
|
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
|
|
|
auto task = makeReference<Task>(OldCopyLogRangeTaskFunc::name, OldCopyLogRangeTaskFunc::version, doneKey, 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyBeginVersion] = BinaryWriter::toValue(beginVersion, Unversioned());
|
2018-04-28 04:54:34 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyEndVersion] = BinaryWriter::toValue(endVersion, Unversioned());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2019-02-13 08:07:17 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2019-02-13 08:07:17 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
state Version endVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyEndVersion], Unversioned());
|
|
|
|
state Reference<TaskFuture> taskFuture = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// Get the bytesWritten parameter from task and atomically add it to the logBytesWritten() property of the DR
|
|
|
|
// config.
|
2018-04-27 08:24:40 +08:00
|
|
|
DRConfig config(task);
|
|
|
|
int64_t bytesWritten = Params.bytesWritten().getOrDefault(task);
|
|
|
|
config.logBytesWritten().atomicOp(tr, bytesWritten, MutationRef::AddValue);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (task->params.find(OldCopyLogRangeTaskFunc::keyNextBeginVersion) != task->params.end()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version nextVersion = BinaryReader::fromStringRef<Version>(
|
2018-04-27 08:24:40 +08:00
|
|
|
task->params[OldCopyLogRangeTaskFunc::keyNextBeginVersion], Unversioned());
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(success(OldCopyLogRangeTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, nextVersion, endVersion, TaskCompletionKey::signal(taskFuture->key))) &&
|
|
|
|
taskBucket->finish(tr, task));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskFuture->set(tr, taskBucket) && taskBucket->finish(tr, task));
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
return Void();
|
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef OldCopyLogRangeTaskFunc::name = "db_copy_log_range"_sr;
|
|
|
|
const Key OldCopyLogRangeTaskFunc::keyNextBeginVersion = "nextBeginVersion"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
REGISTER_TASKFUNC(OldCopyLogRangeTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
struct AbortOldBackupTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
2018-08-11 04:57:10 +08:00
|
|
|
Reference<Task> task) {
|
2020-09-24 05:50:34 +08:00
|
|
|
state DatabaseBackupAgent srcDrAgent(taskBucket->src);
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
2018-03-14 02:21:24 +08:00
|
|
|
state Key tagNameKey;
|
|
|
|
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2018-08-11 04:57:10 +08:00
|
|
|
Key tagPath = srcDrAgent.states.get(task->params[DatabaseBackupAgent::keyConfigLogUid])
|
2020-02-26 07:48:00 +08:00
|
|
|
.pack(BackupAgentBase::keyConfigBackupTag);
|
2018-08-11 04:57:10 +08:00
|
|
|
Optional<Key> tagName = wait(tr->get(tagPath));
|
|
|
|
if (!tagName.present()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
tagNameKey = tagName.get();
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2018-02-21 05:22:31 +08:00
|
|
|
} catch (Error& e) {
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
TraceEvent("DBA_AbortOldBackup").detail("TagName", tagNameKey.printable());
|
2021-07-17 15:11:40 +08:00
|
|
|
wait(srcDrAgent.abortBackup(cx, tagNameKey, PartialBackup::False, AbortOldBackup::True));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
|
|
|
Reference<Task> task) {
|
|
|
|
wait(taskBucket->finish(tr, task));
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
2017-05-26 04:48:44 +08:00
|
|
|
TaskCompletionKey completionKey,
|
2018-08-11 04:57:10 +08:00
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
|
|
|
auto task = makeReference<Task>(AbortOldBackupTaskFunc::name, AbortOldBackupTaskFunc::version, doneKey, 1);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
if (!waitFor) {
|
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-04-27 08:24:40 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2018-08-11 04:57:10 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _execute(cx, tb, fb, task);
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
2018-04-27 08:24:40 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef AbortOldBackupTaskFunc::name = "dr_abort_legacy_backup"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
REGISTER_TASKFUNC(AbortOldBackupTaskFunc);
|
|
|
|
REGISTER_TASKFUNC_ALIAS(AbortOldBackupTaskFunc, db_backup_range);
|
|
|
|
REGISTER_TASKFUNC_ALIAS(AbortOldBackupTaskFunc, db_finish_full_backup);
|
|
|
|
REGISTER_TASKFUNC_ALIAS(AbortOldBackupTaskFunc, db_copy_logs);
|
|
|
|
REGISTER_TASKFUNC_ALIAS(AbortOldBackupTaskFunc, db_finished_full_backup);
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC_ALIAS(AbortOldBackupTaskFunc, db_backup_restorable);
|
2018-04-27 08:24:40 +08:00
|
|
|
REGISTER_TASKFUNC_ALIAS(AbortOldBackupTaskFunc, db_start_full_backup);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
// Upgrade DR from 5.1
|
|
|
|
struct CopyDiffLogsUpgradeTaskFunc : TaskFuncBase {
|
2017-05-26 04:48:44 +08:00
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
2018-02-21 05:22:31 +08:00
|
|
|
state Key logUidValue = task->params[DatabaseBackupAgent::keyConfigLogUid];
|
|
|
|
state Subspace sourceStates =
|
2017-05-26 04:48:44 +08:00
|
|
|
Subspace(databaseBackupPrefixRange.begin).get(BackupAgentBase::keySourceStates).get(logUidValue);
|
2018-02-21 05:22:31 +08:00
|
|
|
state Subspace config =
|
|
|
|
Subspace(databaseBackupPrefixRange.begin).get(BackupAgentBase::keyConfig).get(logUidValue);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(cx, task, CopyDiffLogsUpgradeTaskFunc::name, CopyDiffLogsUpgradeTaskFunc::version));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
// Retrieve backupRanges
|
|
|
|
state Standalone<VectorRef<KeyRangeRef>> backupRanges;
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2018-04-28 04:54:34 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
2018-02-21 05:22:31 +08:00
|
|
|
Future<Void> verified = taskBucket->keepRunning(tr, task);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(verified);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
Optional<Key> backupKeysPacked = wait(tr->get(config.pack(BackupAgentBase::keyConfigBackupRanges)));
|
|
|
|
if (!backupKeysPacked.present()) {
|
2020-10-03 08:44:31 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
BinaryReader br(backupKeysPacked.get(), IncludeVersion());
|
|
|
|
br >> backupRanges;
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2018-08-11 04:57:10 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
wait(tr->onError(e));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// Set destUidValue and versionKey on src side
|
|
|
|
state Key destUidValue(logUidValue);
|
|
|
|
state Reference<ReadYourWritesTransaction> srcTr(new ReadYourWritesTransaction(taskBucket->src));
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
2017-05-26 04:48:44 +08:00
|
|
|
try {
|
|
|
|
srcTr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2018-03-14 02:21:24 +08:00
|
|
|
srcTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
2018-02-21 05:22:31 +08:00
|
|
|
|
2018-03-14 02:21:24 +08:00
|
|
|
state Optional<Value> v = wait(srcTr->get(sourceStates.pack(DatabaseBackupAgent::keyFolderId)));
|
|
|
|
if (v.present() && BinaryReader::fromStringRef<Version>(v.get(), Unversioned()) >
|
2018-02-21 05:22:31 +08:00
|
|
|
BinaryReader::fromStringRef<Version>(
|
|
|
|
task->params[DatabaseBackupAgent::keyFolderId], Unversioned())) {
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
|
2022-09-23 02:36:46 +08:00
|
|
|
if (backupRanges.size() == 1 || isDefaultBackup(backupRanges)) {
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult existingDestUidValues = wait(srcTr->getRange(
|
2019-10-02 09:52:07 +08:00
|
|
|
KeyRangeRef(destUidLookupPrefix, strinc(destUidLookupPrefix)), CLIENT_KNOBS->TOO_MANY));
|
|
|
|
bool found = false;
|
2022-09-23 02:36:46 +08:00
|
|
|
KeyRangeRef targetRange =
|
|
|
|
(backupRanges.size() == 1) ? backupRanges[0] : getDefaultBackupSharedRange();
|
2019-10-02 09:52:07 +08:00
|
|
|
for (auto it : existingDestUidValues) {
|
2022-09-23 02:36:46 +08:00
|
|
|
KeyRange uidRange = BinaryReader::fromStringRef<KeyRange>(
|
|
|
|
it.key.removePrefix(destUidLookupPrefix), IncludeVersion());
|
|
|
|
if (uidRange == targetRange) {
|
2019-10-02 09:52:07 +08:00
|
|
|
if (destUidValue != it.value) {
|
|
|
|
// existing backup/DR is running
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2019-10-02 09:52:07 +08:00
|
|
|
// due to unknown commit result
|
|
|
|
found = true;
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
}
|
2020-04-30 04:50:13 +08:00
|
|
|
if (found) {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
srcTr->set(
|
2022-09-23 02:36:46 +08:00
|
|
|
BinaryWriter::toValue(targetRange, IncludeVersion(ProtocolVersion::withSharedMutations()))
|
2018-02-21 05:22:31 +08:00
|
|
|
.withPrefix(destUidLookupPrefix),
|
|
|
|
destUidValue);
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-04-30 04:50:13 +08:00
|
|
|
Key versionKey = logUidValue.withPrefix(destUidValue).withPrefix(backupLatestVersionsPrefix);
|
2018-04-27 08:24:40 +08:00
|
|
|
srcTr->set(versionKey, task->params[DatabaseBackupAgent::keyBeginVersion]);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(srcTr->commit());
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2018-04-28 04:54:34 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(srcTr->onError(e));
|
2018-02-21 05:22:31 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
|
|
|
|
task->params[BackupAgentBase::destUid] = destUidValue;
|
2018-08-11 04:57:10 +08:00
|
|
|
ASSERT(destUidValue == logUidValue);
|
2018-02-21 05:22:31 +08:00
|
|
|
|
2018-03-14 02:21:24 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2018-03-14 02:21:24 +08:00
|
|
|
|
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
2018-03-14 02:21:24 +08:00
|
|
|
wait(checkTaskVersion(tr, task, CopyDiffLogsUpgradeTaskFunc::name, CopyDiffLogsUpgradeTaskFunc::version));
|
|
|
|
state Reference<TaskFuture> onDone = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
if (task->params[BackupAgentBase::destUid].size() == 0) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_CopyDiffLogsUpgradeTaskFuncAbortInUpgrade").log();
|
2018-03-14 02:21:24 +08:00
|
|
|
wait(success(AbortOldBackupTaskFunc::addTask(tr, taskBucket, task, TaskCompletionKey::signal(onDone))));
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2018-03-14 02:21:24 +08:00
|
|
|
Version beginVersion =
|
2018-04-27 08:24:40 +08:00
|
|
|
BinaryReader::fromStringRef<Version>(task->params[DatabaseBackupAgent::keyBeginVersion], Unversioned());
|
2018-02-21 05:22:31 +08:00
|
|
|
Subspace config = Subspace(databaseBackupPrefixRange.begin)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(BackupAgentBase::keyConfig)
|
2018-04-28 04:54:34 +08:00
|
|
|
.get(task->params[DatabaseBackupAgent::keyConfigLogUid]);
|
2018-03-14 02:21:24 +08:00
|
|
|
tr->set(config.pack(BackupAgentBase::destUid), task->params[BackupAgentBase::destUid]);
|
|
|
|
tr->set(config.pack(BackupAgentBase::keyDrVersion),
|
|
|
|
BinaryWriter::toValue(DatabaseBackupAgent::LATEST_DR_VERSION, Unversioned()));
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(CopyDiffLogsTaskFunc::addTask(
|
2018-03-14 02:21:24 +08:00
|
|
|
tr, taskBucket, task, 0, beginVersion, TaskCompletionKey::signal(onDone))));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
2018-04-28 04:54:34 +08:00
|
|
|
return Void();
|
2018-03-14 02:21:24 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
StringRef getName() const override { return name; };
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
Future<Void> execute(Database cx,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
2018-02-21 05:22:31 +08:00
|
|
|
Reference<Task> task) override {
|
2021-01-26 10:03:09 +08:00
|
|
|
return _execute(cx, tb, fb, task);
|
2018-02-21 05:22:31 +08:00
|
|
|
};
|
2020-10-19 07:46:41 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
2020-10-19 07:46:41 +08:00
|
|
|
Reference<Task> task) override {
|
2021-01-26 10:03:09 +08:00
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef CopyDiffLogsUpgradeTaskFunc::name = "db_copy_diff_logs"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(CopyDiffLogsUpgradeTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-19 07:46:41 +08:00
|
|
|
struct BackupRestorableTaskFunc : TaskFuncBase {
|
2017-05-26 04:48:44 +08:00
|
|
|
static StringRef name;
|
2020-10-19 07:46:41 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
2020-10-19 07:46:41 +08:00
|
|
|
state Subspace sourceStates = Subspace(databaseBackupPrefixRange.begin)
|
2018-02-21 05:22:31 +08:00
|
|
|
.get(BackupAgentBase::keySourceStates)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2020-10-19 07:46:41 +08:00
|
|
|
wait(checkTaskVersion(cx, task, BackupRestorableTaskFunc::name, BackupRestorableTaskFunc::version));
|
2018-08-11 04:57:10 +08:00
|
|
|
state Transaction tr(taskBucket->src);
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2018-08-11 04:57:10 +08:00
|
|
|
tr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
2017-05-26 04:48:44 +08:00
|
|
|
tr.addReadConflictRange(singleKeyRange(sourceStates.pack(DatabaseBackupAgent::keyStateStatus)));
|
2020-09-24 05:50:34 +08:00
|
|
|
tr.set(sourceStates.pack(DatabaseBackupAgent::keyStateStatus),
|
2018-08-11 04:57:10 +08:00
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_RUNNING_DIFFERENTIAL)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
Key versionKey = task->params[DatabaseBackupAgent::keyConfigLogUid]
|
2018-02-21 05:22:31 +08:00
|
|
|
.withPrefix(task->params[BackupAgentBase::destUid])
|
2020-04-30 04:50:13 +08:00
|
|
|
.withPrefix(backupLatestVersionsPrefix);
|
2018-08-11 04:57:10 +08:00
|
|
|
Optional<Key> prevBeginVersion = wait(tr.get(versionKey));
|
2018-03-14 02:21:24 +08:00
|
|
|
if (!prevBeginVersion.present()) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2018-02-21 05:22:31 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyPrevBeginVersion] = prevBeginVersion.get();
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr.commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
2018-04-28 04:54:34 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr.onError(e));
|
2018-02-21 05:22:31 +08:00
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
|
2019-03-01 09:45:00 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
2019-03-01 09:45:00 +08:00
|
|
|
state Subspace conf = Subspace(databaseBackupPrefixRange.begin)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(BackupAgentBase::keyConfig)
|
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2018-02-21 05:22:31 +08:00
|
|
|
state Subspace states = Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keyStates)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(task->params[BackupAgentBase::keyConfigLogUid]);
|
2019-03-01 09:45:00 +08:00
|
|
|
wait(checkTaskVersion(tr, task, BackupRestorableTaskFunc::name, BackupRestorableTaskFunc::version));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2019-03-01 09:45:00 +08:00
|
|
|
state Reference<TaskFuture> onDone = futureBucket->unpack(task->params[Task::reservedTaskParamKeyDone]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
state Optional<Value> stopValue = wait(tr->get(states.pack(DatabaseBackupAgent::keyStateStop)));
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version restoreVersion =
|
|
|
|
stopValue.present() ? BinaryReader::fromStringRef<Version>(stopValue.get(), Unversioned()) : -1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
state Optional<Value> stopWhenDone = wait(tr->get(conf.pack(DatabaseBackupAgent::keyConfigStopWhenDoneKey)));
|
2017-05-26 04:48:44 +08:00
|
|
|
state Reference<TaskFuture> allPartsDone;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
TraceEvent("DBA_Complete")
|
2019-03-19 06:03:43 +08:00
|
|
|
.detail("RestoreVersion", restoreVersion)
|
|
|
|
.detail("Differential", stopWhenDone.present())
|
|
|
|
.detail("LogUID", task->params[BackupAgentBase::keyConfigLogUid]);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// Start the complete task, if differential is not enabled
|
|
|
|
if (stopWhenDone.present()) {
|
2018-02-21 05:22:31 +08:00
|
|
|
// After the Backup completes, clear the backup subspace and update the status
|
|
|
|
wait(success(FinishedFullBackupTaskFunc::addTask(tr, taskBucket, task, TaskCompletionKey::noSignal())));
|
2017-05-26 04:48:44 +08:00
|
|
|
} else { // Start the writing of logs, if differential
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->set(states.pack(DatabaseBackupAgent::keyStateStatus),
|
2020-09-24 05:50:34 +08:00
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_RUNNING_DIFFERENTIAL)));
|
2018-02-21 05:22:31 +08:00
|
|
|
|
|
|
|
allPartsDone = futureBucket->future(tr);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
Version prevBeginVersion = BinaryReader::fromStringRef<Version>(
|
|
|
|
task->params[DatabaseBackupAgent::keyPrevBeginVersion], Unversioned());
|
|
|
|
wait(success(CopyDiffLogsTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, prevBeginVersion, restoreVersion, TaskCompletionKey::joinWith(allPartsDone))));
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
// After the Backup completes, clear the backup subspace and update the status
|
|
|
|
wait(success(FinishedFullBackupTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, TaskCompletionKey::noSignal(), allPartsDone)));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-03-14 02:21:24 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
ACTOR static Future<Key> addTask(Reference<ReadYourWritesTransaction> tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<Task> parentTask,
|
|
|
|
TaskCompletionKey completionKey,
|
2018-08-11 04:57:10 +08:00
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>()) {
|
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2018-02-21 05:22:31 +08:00
|
|
|
auto task = makeReference<Task>(BackupRestorableTaskFunc::name, BackupRestorableTaskFunc::version, doneKey);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
copyDefaultParameters(parentTask, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
if (!waitFor) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-04-27 08:24:40 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2018-08-11 04:57:10 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-04-27 08:24:40 +08:00
|
|
|
parentTask->params[Task::reservedTaskParamValidKey],
|
2018-08-11 04:57:10 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
2018-02-21 05:22:31 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _execute(cx, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef BackupRestorableTaskFunc::name = "dr_backup_restorable"_sr;
|
2018-04-27 08:24:40 +08:00
|
|
|
REGISTER_TASKFUNC(BackupRestorableTaskFunc);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
struct StartFullBackupTaskFunc : TaskFuncBase {
|
|
|
|
static StringRef name;
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr uint32_t version = 1;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-10-03 08:44:31 +08:00
|
|
|
ACTOR static Future<Void> _execute(Database cx,
|
2017-05-26 04:48:44 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
2018-02-21 05:22:31 +08:00
|
|
|
state Key logUidValue = task->params[DatabaseBackupAgent::keyConfigLogUid];
|
|
|
|
state Subspace sourceStates =
|
|
|
|
Subspace(databaseBackupPrefixRange.begin).get(BackupAgentBase::keySourceStates).get(logUidValue);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkTaskVersion(cx, task, StartFullBackupTaskFunc::name, StartFullBackupTaskFunc::version));
|
2018-02-21 05:22:31 +08:00
|
|
|
state Key destUidValue(logUidValue);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Standalone<VectorRef<KeyRangeRef>> backupRanges =
|
|
|
|
BinaryReader::fromStringRef<Standalone<VectorRef<KeyRangeRef>>>(
|
|
|
|
task->params[DatabaseBackupAgent::keyConfigBackupRanges], IncludeVersion());
|
|
|
|
state Key beginVersionKey;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2019-03-01 09:45:00 +08:00
|
|
|
state Reference<ReadYourWritesTransaction> srcTr(new ReadYourWritesTransaction(taskBucket->src));
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
srcTr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
srcTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
// Initialize destUid
|
2022-09-23 02:36:46 +08:00
|
|
|
if (backupRanges.size() == 1 || isDefaultBackup(backupRanges)) {
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult existingDestUidValues = wait(srcTr->getRange(
|
2019-10-02 09:52:07 +08:00
|
|
|
KeyRangeRef(destUidLookupPrefix, strinc(destUidLookupPrefix)), CLIENT_KNOBS->TOO_MANY));
|
2022-09-23 02:36:46 +08:00
|
|
|
KeyRangeRef targetRange =
|
|
|
|
(backupRanges.size() == 1) ? backupRanges[0] : getDefaultBackupSharedRange();
|
2020-06-16 18:59:47 +08:00
|
|
|
bool found = false;
|
2019-10-02 09:52:07 +08:00
|
|
|
for (auto it : existingDestUidValues) {
|
2022-09-23 02:36:46 +08:00
|
|
|
KeyRange uidRange = BinaryReader::fromStringRef<KeyRange>(
|
|
|
|
it.key.removePrefix(destUidLookupPrefix), IncludeVersion());
|
|
|
|
if (uidRange == targetRange) {
|
2020-10-19 07:46:41 +08:00
|
|
|
destUidValue = it.value;
|
2019-10-02 09:52:07 +08:00
|
|
|
found = true;
|
2022-09-28 06:25:30 +08:00
|
|
|
CODE_PROBE(targetRange == getDefaultBackupSharedRange(),
|
|
|
|
"DR mutation sharing with default backup");
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
2019-03-05 07:58:36 +08:00
|
|
|
if (!found) {
|
2019-05-11 05:01:52 +08:00
|
|
|
destUidValue = BinaryWriter::toValue(deterministicRandom()->randomUniqueID(), Unversioned());
|
2022-09-23 02:36:46 +08:00
|
|
|
srcTr->set(
|
|
|
|
BinaryWriter::toValue(targetRange, IncludeVersion(ProtocolVersion::withSharedMutations()))
|
|
|
|
.withPrefix(destUidLookupPrefix),
|
|
|
|
destUidValue);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
}
|
2019-03-01 09:45:00 +08:00
|
|
|
|
|
|
|
Version bVersion = wait(srcTr->getReadVersion());
|
|
|
|
beginVersionKey = BinaryWriter::toValue(bVersion, Unversioned());
|
|
|
|
|
|
|
|
state Key versionKey = logUidValue.withPrefix(destUidValue).withPrefix(backupLatestVersionsPrefix);
|
|
|
|
Optional<Key> versionRecord = wait(srcTr->get(versionKey));
|
2020-04-30 04:50:13 +08:00
|
|
|
if (!versionRecord.present()) {
|
|
|
|
srcTr->set(versionKey, beginVersionKey);
|
2019-03-01 09:45:00 +08:00
|
|
|
}
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
task->params[BackupAgentBase::destUid] = destUidValue;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
wait(srcTr->commit());
|
|
|
|
break;
|
2017-12-01 09:37:00 +08:00
|
|
|
} catch (Error& e) {
|
2020-07-05 02:31:23 +08:00
|
|
|
wait(srcTr->onError(e));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
loop {
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
2021-03-11 02:06:03 +08:00
|
|
|
try {
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
state Future<Void> verified = taskBucket->keepRunning(tr, task);
|
2018-02-21 05:22:31 +08:00
|
|
|
wait(verified);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Set destUid at destination side
|
|
|
|
state Subspace config =
|
|
|
|
Subspace(databaseBackupPrefixRange.begin).get(BackupAgentBase::keyConfig).get(logUidValue);
|
|
|
|
tr->set(config.pack(BackupAgentBase::destUid), task->params[BackupAgentBase::destUid]);
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
// Use existing beginVersion if we already have one
|
|
|
|
Optional<Key> backupStartVersion = wait(tr->get(config.pack(BackupAgentBase::backupStartVersion)));
|
|
|
|
if (backupStartVersion.present()) {
|
|
|
|
beginVersionKey = backupStartVersion.get();
|
2021-03-11 02:06:03 +08:00
|
|
|
} else {
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->set(config.pack(BackupAgentBase::backupStartVersion), beginVersionKey);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
task->params[BackupAgentBase::keyBeginVersion] = beginVersionKey;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(tr->commit());
|
|
|
|
break;
|
2017-12-01 09:37:00 +08:00
|
|
|
} catch (Error& e) {
|
2022-02-25 04:25:52 +08:00
|
|
|
TraceEvent("SetDestUidOrBeginVersionError").errorUnsuppressed(e);
|
2017-05-26 04:48:44 +08:00
|
|
|
wait(tr->onError(e));
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
state Reference<ReadYourWritesTransaction> srcTr2(new ReadYourWritesTransaction(taskBucket->src));
|
2021-03-11 02:06:03 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2018-02-21 05:22:31 +08:00
|
|
|
srcTr2->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2020-07-05 02:31:23 +08:00
|
|
|
srcTr2->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
state Optional<Value> v = wait(srcTr2->get(sourceStates.pack(DatabaseBackupAgent::keyFolderId)));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
if (v.present() && BinaryReader::fromStringRef<Version>(v.get(), Unversioned()) >=
|
|
|
|
BinaryReader::fromStringRef<Version>(
|
|
|
|
task->params[DatabaseBackupAgent::keyFolderId], Unversioned()))
|
|
|
|
return Void();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
srcTr2->set(Subspace(databaseBackupPrefixRange.begin)
|
|
|
|
.get(BackupAgentBase::keySourceTagName)
|
2018-08-11 04:57:10 +08:00
|
|
|
.pack(task->params[BackupAgentBase::keyTagName]),
|
2017-05-26 04:48:44 +08:00
|
|
|
logUidValue);
|
|
|
|
srcTr2->set(sourceStates.pack(DatabaseBackupAgent::keyFolderId),
|
2018-02-21 05:22:31 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyFolderId]);
|
2017-05-26 04:48:44 +08:00
|
|
|
srcTr2->set(sourceStates.pack(DatabaseBackupAgent::keyStateStatus),
|
2020-09-24 05:50:34 +08:00
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_RUNNING)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Key destPath = destUidValue.withPrefix(backupLogKeys.begin);
|
|
|
|
// Start logging the mutations for the specified ranges of the tag
|
2019-02-13 08:07:17 +08:00
|
|
|
for (auto& backupRange : backupRanges) {
|
|
|
|
srcTr2->set(logRangesEncodeKey(backupRange.begin,
|
|
|
|
BinaryReader::fromStringRef<UID>(destUidValue, Unversioned())),
|
|
|
|
logRangesEncodeValue(backupRange.end, destPath));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(srcTr2->commit());
|
2021-03-11 02:06:03 +08:00
|
|
|
break;
|
2017-12-01 09:37:00 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(srcTr2->onError(e));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
state Reference<ReadYourWritesTransaction> srcTr3(new ReadYourWritesTransaction(taskBucket->src));
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
srcTr3->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2020-11-07 15:50:55 +08:00
|
|
|
srcTr3->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
srcTr3->atomicOp(metadataVersionKey, metadataVersionRequiredValue, MutationRef::SetVersionstampedValue);
|
|
|
|
|
|
|
|
wait(srcTr3->commit());
|
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
|
|
|
wait(srcTr3->onError(e));
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
ACTOR static Future<Void> _finish(Reference<ReadYourWritesTransaction> tr,
|
2020-10-03 08:44:31 +08:00
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Reference<FutureBucket> futureBucket,
|
2021-01-26 10:03:09 +08:00
|
|
|
Reference<Task> task) {
|
2018-02-21 05:22:31 +08:00
|
|
|
state Key logUidValue = task->params[BackupAgentBase::keyConfigLogUid];
|
|
|
|
state Subspace states =
|
|
|
|
Subspace(databaseBackupPrefixRange.begin).get(BackupAgentBase::keyStates).get(logUidValue);
|
|
|
|
state Subspace config =
|
|
|
|
Subspace(databaseBackupPrefixRange.begin).get(BackupAgentBase::keyConfig).get(logUidValue);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Version beginVersion =
|
|
|
|
BinaryReader::fromStringRef<Version>(task->params[BackupAgentBase::keyBeginVersion], Unversioned());
|
2020-02-26 07:48:00 +08:00
|
|
|
state Standalone<VectorRef<KeyRangeRef>> backupRanges =
|
2017-05-26 04:48:44 +08:00
|
|
|
BinaryReader::fromStringRef<Standalone<VectorRef<KeyRangeRef>>>(
|
2020-05-23 08:14:21 +08:00
|
|
|
task->params[DatabaseBackupAgent::keyConfigBackupRanges], IncludeVersion());
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->set(logUidValue.withPrefix(applyMutationsBeginRange.begin),
|
2017-05-26 04:48:44 +08:00
|
|
|
BinaryWriter::toValue(beginVersion, Unversioned()));
|
|
|
|
tr->set(logUidValue.withPrefix(applyMutationsEndRange.begin),
|
|
|
|
BinaryWriter::toValue(beginVersion, Unversioned()));
|
2020-09-24 05:50:34 +08:00
|
|
|
tr->set(states.pack(DatabaseBackupAgent::keyStateStatus),
|
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_RUNNING)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
state Reference<TaskFuture> kvBackupRangeComplete = futureBucket->future(tr);
|
|
|
|
state Reference<TaskFuture> kvBackupComplete = futureBucket->future(tr);
|
|
|
|
state int rangeCount = 0;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (task->params[DatabaseBackupAgent::keyDatabasesInSync] != std::string("t")) {
|
|
|
|
for (; rangeCount < backupRanges.size(); ++rangeCount) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(success(BackupRangeTaskFunc::addTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2019-02-13 08:07:17 +08:00
|
|
|
backupRanges[rangeCount].begin,
|
|
|
|
backupRanges[rangeCount].end,
|
|
|
|
TaskCompletionKey::joinWith(kvBackupRangeComplete))));
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
} else {
|
2017-05-26 04:48:44 +08:00
|
|
|
kvBackupRangeComplete->set(tr, taskBucket);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// After the BackupRangeTask completes, set the stop key which will stop the BackupLogsTask
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(FinishFullBackupTaskFunc::addTask(
|
|
|
|
tr, taskBucket, task, TaskCompletionKey::noSignal(), kvBackupRangeComplete)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// Backup the logs which will create BackupLogRange tasks
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(CopyLogsTaskFunc::addTask(
|
2018-06-07 04:05:53 +08:00
|
|
|
tr, taskBucket, task, 0, beginVersion, TaskCompletionKey::joinWith(kvBackupComplete))));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// After the Backup completes, clear the backup subspace and update the status
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(
|
2018-02-21 05:22:31 +08:00
|
|
|
BackupRestorableTaskFunc::addTask(tr, taskBucket, task, TaskCompletionKey::noSignal(), kvBackupComplete)));
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(taskBucket->finish(tr, task));
|
2020-07-05 02:31:23 +08:00
|
|
|
return Void();
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2021-03-31 08:17:47 +08:00
|
|
|
ACTOR static Future<Key> addTask(
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> taskBucket,
|
|
|
|
Key logUid,
|
|
|
|
Key backupUid,
|
|
|
|
Key keyAddPrefix,
|
|
|
|
Key keyRemovePrefix,
|
|
|
|
Key keyConfigBackupRanges,
|
|
|
|
Key tagName,
|
|
|
|
TaskCompletionKey completionKey,
|
|
|
|
Reference<TaskFuture> waitFor = Reference<TaskFuture>(),
|
|
|
|
DatabaseBackupAgent::PreBackupAction backupAction = DatabaseBackupAgent::PreBackupAction::VERIFY) {
|
2017-05-26 04:48:44 +08:00
|
|
|
Key doneKey = wait(completionKey.get(tr, taskBucket));
|
2020-11-07 15:50:55 +08:00
|
|
|
auto task = makeReference<Task>(StartFullBackupTaskFunc::name, StartFullBackupTaskFunc::version, doneKey);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
task->params[BackupAgentBase::keyFolderId] = backupUid;
|
2017-05-26 04:48:44 +08:00
|
|
|
task->params[BackupAgentBase::keyConfigLogUid] = logUid;
|
|
|
|
task->params[DatabaseBackupAgent::keyAddPrefix] = keyAddPrefix;
|
|
|
|
task->params[DatabaseBackupAgent::keyRemovePrefix] = keyRemovePrefix;
|
|
|
|
task->params[BackupAgentBase::keyConfigBackupRanges] = keyConfigBackupRanges;
|
|
|
|
task->params[BackupAgentBase::keyTagName] = tagName;
|
|
|
|
task->params[DatabaseBackupAgent::keyDatabasesInSync] =
|
2022-09-20 02:35:58 +08:00
|
|
|
backupAction == DatabaseBackupAgent::PreBackupAction::NONE ? "t"_sr : "f"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!waitFor) {
|
2018-04-27 08:24:40 +08:00
|
|
|
return taskBucket->addTask(tr,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
Subspace(databaseBackupPrefixRange.begin)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(BackupAgentBase::keyConfig)
|
2020-07-05 02:31:23 +08:00
|
|
|
.get(logUid)
|
2018-08-11 04:57:10 +08:00
|
|
|
.pack(BackupAgentBase::keyFolderId),
|
|
|
|
task->params[BackupAgentBase::keyFolderId]);
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitFor->onSetAddTask(tr,
|
2017-05-26 04:48:44 +08:00
|
|
|
taskBucket,
|
2021-03-11 02:06:03 +08:00
|
|
|
task,
|
2018-08-11 04:57:10 +08:00
|
|
|
Subspace(databaseBackupPrefixRange.begin)
|
2017-05-26 04:48:44 +08:00
|
|
|
.get(BackupAgentBase::keyConfig)
|
2020-07-05 02:31:23 +08:00
|
|
|
.get(logUid)
|
2018-08-11 04:57:10 +08:00
|
|
|
.pack(BackupAgentBase::keyFolderId),
|
|
|
|
task->params[BackupAgentBase::keyFolderId]));
|
2022-09-20 02:35:58 +08:00
|
|
|
return "OnSetAddTask"_sr;
|
2021-03-11 02:06:03 +08:00
|
|
|
}
|
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
StringRef getName() const override { return name; };
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> execute(Database cx,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _execute(cx, tb, fb, task);
|
2017-05-26 04:48:44 +08:00
|
|
|
};
|
2021-01-26 10:03:09 +08:00
|
|
|
Future<Void> finish(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Reference<TaskBucket> tb,
|
|
|
|
Reference<FutureBucket> fb,
|
|
|
|
Reference<Task> task) override {
|
|
|
|
return _finish(tr, tb, fb, task);
|
2021-03-11 02:06:03 +08:00
|
|
|
};
|
|
|
|
};
|
2022-09-20 02:35:58 +08:00
|
|
|
StringRef StartFullBackupTaskFunc::name = "dr_start_full_backup"_sr;
|
2017-05-26 04:48:44 +08:00
|
|
|
REGISTER_TASKFUNC(StartFullBackupTaskFunc);
|
|
|
|
} // namespace dbBackup
|
|
|
|
|
2019-03-02 06:49:04 +08:00
|
|
|
std::set<std::string> getDRAgentsIds(StatusObjectReader statusObj, const char* context) {
|
|
|
|
std::set<std::string> drBackupAgents;
|
|
|
|
try {
|
|
|
|
StatusObjectReader statusObjLayers;
|
|
|
|
statusObj.get("cluster.layers", statusObjLayers);
|
|
|
|
StatusObjectReader instances;
|
|
|
|
std::string path = format("%s.instances", context);
|
|
|
|
if (statusObjLayers.tryGet(path, instances)) {
|
|
|
|
for (auto itr : instances.obj()) {
|
|
|
|
drBackupAgents.insert(itr.first);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
} catch (std::runtime_error& e) {
|
|
|
|
TraceEvent(SevWarn, "DBA_GetDRAgentsIdsFail").detail("Error", e.what());
|
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
return drBackupAgents;
|
|
|
|
}
|
|
|
|
|
|
|
|
std::string getDRMutationStreamId(StatusObjectReader statusObj, const char* context, Key tagName) {
|
|
|
|
try {
|
|
|
|
StatusObjectReader statusObjLayers;
|
|
|
|
statusObj.get("cluster.layers", statusObjLayers);
|
|
|
|
StatusObjectReader tags;
|
|
|
|
std::string path = format("%s.tags", context);
|
|
|
|
if (statusObjLayers.tryGet(path, tags)) {
|
|
|
|
for (auto itr : tags.obj()) {
|
|
|
|
if (itr.first == tagName.toString()) {
|
|
|
|
JSONDoc tag(itr.second);
|
|
|
|
return tag["mutation_stream_id"].get_str();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
2021-08-08 15:03:25 +08:00
|
|
|
TraceEvent(SevWarn, "DBA_TagNotPresentInStatus").detail("Tag", tagName).detail("Context", context);
|
2019-03-02 06:49:04 +08:00
|
|
|
throw backup_error();
|
|
|
|
} catch (std::runtime_error& e) {
|
|
|
|
TraceEvent(SevWarn, "DBA_GetDRMutationStreamIdFail").detail("Error", e.what());
|
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
bool getLockedStatus(StatusObjectReader statusObj) {
|
|
|
|
try {
|
|
|
|
StatusObjectReader statusObjCluster = statusObj["cluster"].get_obj();
|
2022-01-11 12:18:25 +08:00
|
|
|
return statusObjCluster["database_lock_state.locked"].get_bool();
|
2019-03-02 06:49:04 +08:00
|
|
|
} catch (std::runtime_error& e) {
|
|
|
|
TraceEvent(SevWarn, "DBA_GetLockedStatusFail").detail("Error", e.what());
|
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
void checkAtomicSwitchOverConfig(StatusObjectReader srcStatus, StatusObjectReader destStatus, Key tagName) {
|
|
|
|
|
|
|
|
try {
|
|
|
|
// Check if src is unlocked and dest is locked
|
|
|
|
if (getLockedStatus(srcStatus) != false) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent(SevWarn, "DBA_AtomicSwitchOverSrcLocked").log();
|
2019-03-02 06:49:04 +08:00
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
if (getLockedStatus(destStatus) != true) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent(SevWarn, "DBA_AtomicSwitchOverDestUnlocked").log();
|
2019-03-02 06:49:04 +08:00
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
// Check if mutation-stream-id matches
|
|
|
|
if (getDRMutationStreamId(srcStatus, "dr_backup", tagName) !=
|
|
|
|
getDRMutationStreamId(destStatus, "dr_backup_dest", tagName)) {
|
|
|
|
TraceEvent(SevWarn, "DBA_AtomicSwitchOverMutationIdMismatch")
|
|
|
|
.detail("SourceMutationId", getDRMutationStreamId(srcStatus, "dr_backup", tagName))
|
|
|
|
.detail("DestMutationId", getDRMutationStreamId(destStatus, "dr_back_dest", tagName));
|
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
// Check if there are agents set up with src as its destination cluster and dest as its source cluster
|
|
|
|
auto srcDRAgents = getDRAgentsIds(srcStatus, "dr_backup_dest");
|
|
|
|
auto destDRAgents = getDRAgentsIds(destStatus, "dr_backup");
|
|
|
|
std::set<std::string> intersectingAgents;
|
|
|
|
std::set_intersection(srcDRAgents.begin(),
|
|
|
|
srcDRAgents.end(),
|
|
|
|
destDRAgents.begin(),
|
|
|
|
destDRAgents.end(),
|
|
|
|
std::inserter(intersectingAgents, intersectingAgents.begin()));
|
|
|
|
if (intersectingAgents.empty()) {
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent(SevWarn, "DBA_SwitchOverPossibleDRAgentsIncorrectSetup").log();
|
2019-03-02 06:49:04 +08:00
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
} catch (std::runtime_error& e) {
|
|
|
|
TraceEvent(SevWarn, "DBA_UnableToCheckAtomicSwitchOverConfig").detail("RunTimeError", e.what());
|
|
|
|
throw backup_error();
|
|
|
|
}
|
|
|
|
return;
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
class DatabaseBackupAgentImpl {
|
|
|
|
public:
|
2020-10-03 08:44:31 +08:00
|
|
|
static constexpr int MAX_RESTORABLE_FILE_METASECTION_BYTES = 1024 * 8;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
ACTOR static Future<Void> waitUpgradeToLatestDrVersion(DatabaseBackupAgent* backupAgent, Database cx, Key tagName) {
|
|
|
|
state UID logUid = wait(backupAgent->getLogUid(cx, tagName));
|
|
|
|
state Key drVersionKey = backupAgent->config.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyDrVersion);
|
|
|
|
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DRU_WatchLatestDrVersion")
|
|
|
|
.detail("DrVersionKey", drVersionKey.printable())
|
|
|
|
.detail("LogUid", BinaryWriter::toValue(logUid, Unversioned()).printable());
|
2018-04-27 08:24:40 +08:00
|
|
|
|
|
|
|
loop {
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
|
|
|
|
2018-04-28 04:54:34 +08:00
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
Optional<Value> drVersion = wait(tr->get(drVersionKey));
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DRU_VersionCheck")
|
|
|
|
.detail("Current",
|
|
|
|
drVersion.present() ? BinaryReader::fromStringRef<int>(drVersion.get(), Unversioned())
|
|
|
|
: -1)
|
|
|
|
.detail("Expected", DatabaseBackupAgent::LATEST_DR_VERSION)
|
|
|
|
.detail("LogUid", BinaryWriter::toValue(logUid, Unversioned()).printable());
|
2018-04-28 04:54:34 +08:00
|
|
|
if (drVersion.present() && BinaryReader::fromStringRef<int>(drVersion.get(), Unversioned()) ==
|
|
|
|
DatabaseBackupAgent::LATEST_DR_VERSION) {
|
|
|
|
return Void();
|
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
|
2018-04-28 04:54:34 +08:00
|
|
|
state Future<Void> watchDrVersionFuture = tr->watch(drVersionKey);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
|
|
|
wait(watchDrVersionFuture);
|
2018-04-28 04:54:34 +08:00
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2018-04-28 04:54:34 +08:00
|
|
|
}
|
2018-04-27 08:24:40 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
// This method will return the final status of the backup
|
2020-09-24 05:50:34 +08:00
|
|
|
ACTOR static Future<EBackupState> waitBackup(DatabaseBackupAgent* backupAgent,
|
|
|
|
Database cx,
|
|
|
|
Key tagName,
|
2021-07-05 07:27:26 +08:00
|
|
|
StopWhenDone stopWhenDone) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state std::string backTrace;
|
|
|
|
state UID logUid = wait(backupAgent->getLogUid(cx, tagName));
|
|
|
|
state Key statusKey = backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyStateStatus);
|
|
|
|
|
|
|
|
loop {
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
|
|
|
|
try {
|
2020-09-24 05:50:34 +08:00
|
|
|
state EBackupState status = wait(backupAgent->getStateValue(tr, logUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Break, if no longer runnable
|
2020-09-24 05:50:34 +08:00
|
|
|
if (!DatabaseBackupAgent::isRunnable(status) || EBackupState::STATE_PARTIALLY_ABORTED == status) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
// Break, if in differential mode (restorable) and stopWhenDone is not enabled
|
2020-09-24 05:50:34 +08:00
|
|
|
if ((!stopWhenDone) && (EBackupState::STATE_RUNNING_DIFFERENTIAL == status)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
state Future<Void> watchFuture = tr->watch(statusKey);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
|
|
|
wait(watchFuture);
|
2017-05-26 04:48:44 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// This method will return the final status of the backup
|
2020-09-24 05:50:34 +08:00
|
|
|
ACTOR static Future<EBackupState> waitSubmitted(DatabaseBackupAgent* backupAgent, Database cx, Key tagName) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state UID logUid = wait(backupAgent->getLogUid(cx, tagName));
|
|
|
|
state Key statusKey = backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyStateStatus);
|
|
|
|
|
|
|
|
loop {
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
|
|
|
|
try {
|
2020-09-24 05:50:34 +08:00
|
|
|
state EBackupState status = wait(backupAgent->getStateValue(tr, logUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Break, if no longer runnable
|
2020-09-24 05:50:34 +08:00
|
|
|
if (EBackupState::STATE_SUBMITTED != status) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return status;
|
|
|
|
}
|
|
|
|
|
|
|
|
state Future<Void> watchFuture = tr->watch(statusKey);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
|
|
|
wait(watchFuture);
|
2017-05-26 04:48:44 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR static Future<Void> submitBackup(DatabaseBackupAgent* backupAgent,
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Key tagName,
|
|
|
|
Standalone<VectorRef<KeyRangeRef>> backupRanges,
|
2021-07-05 07:27:26 +08:00
|
|
|
StopWhenDone stopWhenDone,
|
2017-05-26 04:48:44 +08:00
|
|
|
Key addPrefix,
|
|
|
|
Key removePrefix,
|
2021-07-05 07:27:26 +08:00
|
|
|
LockDB lockDB,
|
2021-03-31 08:17:47 +08:00
|
|
|
DatabaseBackupAgent::PreBackupAction backupAction) {
|
2019-05-11 05:01:52 +08:00
|
|
|
state UID logUid = deterministicRandom()->randomUniqueID();
|
2017-05-26 04:48:44 +08:00
|
|
|
state Key logUidValue = BinaryWriter::toValue(logUid, Unversioned());
|
|
|
|
state UID logUidCurrent = wait(backupAgent->getLogUid(tr, tagName));
|
|
|
|
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2020-02-26 04:35:24 +08:00
|
|
|
// This commit must happen on the first proxy to ensure that the applier has flushed all mutations from previous
|
|
|
|
// DRs
|
|
|
|
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// We will use the global status for now to ensure that multiple backups do not start place with different tags
|
2020-09-24 05:50:34 +08:00
|
|
|
state EBackupState status = wait(backupAgent->getStateValue(tr, logUidCurrent));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
if (DatabaseBackupAgent::isRunnable(status)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
throw backup_duplicate();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (logUidCurrent.isValid()) {
|
|
|
|
logUid = logUidCurrent;
|
|
|
|
logUidValue = BinaryWriter::toValue(logUid, Unversioned());
|
|
|
|
}
|
|
|
|
|
|
|
|
Optional<Key> v = wait(tr->get(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId)));
|
|
|
|
Version uidVersion = 0;
|
|
|
|
if (v.present())
|
|
|
|
uidVersion = BinaryReader::fromStringRef<Version>(v.get(), Unversioned()) + 1;
|
|
|
|
state Standalone<StringRef> backupUid = BinaryWriter::toValue(uidVersion, Unversioned());
|
|
|
|
|
|
|
|
KeyRangeMap<int> backupRangeSet;
|
|
|
|
for (auto& backupRange : backupRanges) {
|
|
|
|
backupRangeSet.insert(backupRange, 1);
|
|
|
|
}
|
|
|
|
|
|
|
|
backupRangeSet.coalesce(allKeys);
|
|
|
|
backupRanges = Standalone<VectorRef<KeyRangeRef>>();
|
|
|
|
|
|
|
|
for (auto& backupRange : backupRangeSet.ranges()) {
|
|
|
|
if (backupRange.value()) {
|
|
|
|
backupRanges.push_back_deep(backupRanges.arena(), backupRange.range());
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-03-31 08:17:47 +08:00
|
|
|
if (backupAction == DatabaseBackupAgent::PreBackupAction::VERIFY) {
|
2017-05-26 04:48:44 +08:00
|
|
|
// Make sure all of the ranges are empty before we backup into them.
|
2021-05-04 04:14:16 +08:00
|
|
|
state std::vector<Future<RangeResult>> backupIntoResults;
|
2017-05-26 04:48:44 +08:00
|
|
|
for (auto& backupRange : backupRanges) {
|
|
|
|
backupIntoResults.push_back(
|
|
|
|
tr->getRange(backupRange.removePrefix(removePrefix).withPrefix(addPrefix), 1));
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(waitForAll(backupIntoResults));
|
2017-05-26 04:48:44 +08:00
|
|
|
for (auto result : backupIntoResults) {
|
|
|
|
if (result.get().size() > 0) {
|
|
|
|
// One of the ranges we will be backing up into has pre-existing data.
|
|
|
|
throw restore_destination_not_empty();
|
|
|
|
}
|
|
|
|
}
|
2021-03-31 08:17:47 +08:00
|
|
|
} else if (backupAction == DatabaseBackupAgent::PreBackupAction::CLEAR) {
|
|
|
|
// Clear out all ranges before we backup into them.
|
|
|
|
for (auto& backupRange : backupRanges) {
|
|
|
|
tr->clear(backupRange.removePrefix(removePrefix).withPrefix(addPrefix));
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
// Clear the backup ranges for the tag
|
|
|
|
tr->clear(backupAgent->config.get(logUidValue).range());
|
|
|
|
tr->clear(backupAgent->states.get(logUidValue).range());
|
|
|
|
tr->clear(backupAgent->errors.range());
|
|
|
|
|
|
|
|
tr->set(backupAgent->tagNames.pack(tagName), logUidValue);
|
|
|
|
|
|
|
|
// Clear DRConfig for this UID, which unfortunately only contains some newer vars and not the stuff below.
|
|
|
|
DRConfig(logUid).clear(tr);
|
2018-04-27 08:24:40 +08:00
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyDrVersion),
|
|
|
|
BinaryWriter::toValue(DatabaseBackupAgent::LATEST_DR_VERSION, Unversioned()));
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyAddPrefix), addPrefix);
|
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyRemovePrefix), removePrefix);
|
|
|
|
tr->set(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyConfigBackupTag), tagName);
|
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyConfigLogUid), logUidValue);
|
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId), backupUid);
|
|
|
|
tr->set(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId),
|
|
|
|
backupUid); // written to config and states because it's also used by abort
|
2020-05-23 08:14:21 +08:00
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyConfigBackupRanges),
|
|
|
|
BinaryWriter::toValue(backupRanges, IncludeVersion(ProtocolVersion::withDRBackupRanges())));
|
2020-09-24 05:50:34 +08:00
|
|
|
tr->set(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyStateStatus),
|
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_SUBMITTED)));
|
2017-05-26 04:48:44 +08:00
|
|
|
if (stopWhenDone) {
|
|
|
|
tr->set(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyConfigStopWhenDoneKey),
|
|
|
|
StringRef());
|
|
|
|
}
|
|
|
|
|
|
|
|
int64_t startCount = 0;
|
|
|
|
state Key mapPrefix = logUidValue.withPrefix(applyMutationsKeyVersionMapRange.begin);
|
2022-09-17 06:55:54 +08:00
|
|
|
Key mapEnd = allKeys.end.withPrefix(mapPrefix);
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->set(logUidValue.withPrefix(applyMutationsAddPrefixRange.begin), addPrefix);
|
|
|
|
tr->set(logUidValue.withPrefix(applyMutationsRemovePrefixRange.begin), removePrefix);
|
|
|
|
tr->set(logUidValue.withPrefix(applyMutationsKeyVersionCountRange.begin), StringRef((uint8_t*)&startCount, 8));
|
|
|
|
tr->clear(KeyRangeRef(mapPrefix, mapEnd));
|
|
|
|
|
|
|
|
state Version readVersion = invalidVersion;
|
2021-03-31 08:17:47 +08:00
|
|
|
if (backupAction == DatabaseBackupAgent::PreBackupAction::NONE) {
|
2017-05-26 04:48:44 +08:00
|
|
|
Transaction readTransaction(backupAgent->taskBucket->src);
|
|
|
|
readTransaction.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
Version _ = wait(readTransaction.getReadVersion());
|
|
|
|
readVersion = _;
|
|
|
|
}
|
|
|
|
tr->set(mapPrefix, BinaryWriter::toValue<Version>(readVersion, Unversioned()));
|
|
|
|
|
|
|
|
Key taskKey = wait(dbBackup::StartFullBackupTaskFunc::addTask(
|
|
|
|
tr,
|
|
|
|
backupAgent->taskBucket,
|
|
|
|
logUidValue,
|
|
|
|
backupUid,
|
2020-05-23 08:14:21 +08:00
|
|
|
addPrefix,
|
|
|
|
removePrefix,
|
|
|
|
BinaryWriter::toValue(backupRanges, IncludeVersion(ProtocolVersion::withDRBackupRanges())),
|
|
|
|
tagName,
|
|
|
|
TaskCompletionKey::noSignal(),
|
|
|
|
Reference<TaskFuture>(),
|
2021-03-31 08:17:47 +08:00
|
|
|
backupAction));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if (lockDB)
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(lockDatabase(tr, logUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
else
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(checkDatabaseLock(tr, logUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent("DBA_Submit")
|
|
|
|
.detail("LogUid", logUid)
|
|
|
|
.detail("Lock", lockDB)
|
|
|
|
.detail("LogUID", logUidValue)
|
|
|
|
.detail("Tag", tagName)
|
|
|
|
.detail("Key", backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId))
|
|
|
|
.detail("MapPrefix", mapPrefix);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR static Future<Void> unlockBackup(DatabaseBackupAgent* backupAgent,
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Key tagName) {
|
|
|
|
UID logUid = wait(backupAgent->getLogUid(tr, tagName));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(unlockDatabase(tr, logUid));
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent("DBA_Unlock").detail("Tag", tagName);
|
2017-05-26 04:48:44 +08:00
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2019-03-05 07:58:36 +08:00
|
|
|
ACTOR static Future<Void> atomicSwitchover(DatabaseBackupAgent* backupAgent,
|
|
|
|
Database dest,
|
|
|
|
Key tagName,
|
|
|
|
Standalone<VectorRef<KeyRangeRef>> backupRanges,
|
|
|
|
Key addPrefix,
|
|
|
|
Key removePrefix,
|
2021-07-05 07:27:26 +08:00
|
|
|
ForceAction forceAction) {
|
2017-06-01 07:23:37 +08:00
|
|
|
state DatabaseBackupAgent drAgent(dest);
|
2017-05-26 04:48:44 +08:00
|
|
|
state UID destlogUid = wait(backupAgent->getLogUid(dest, tagName));
|
2020-09-24 05:50:34 +08:00
|
|
|
state EBackupState status = wait(backupAgent->getStateValue(dest, destlogUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_SwitchoverStart").detail("Status", status);
|
2020-09-24 05:50:34 +08:00
|
|
|
if (status != EBackupState::STATE_RUNNING_DIFFERENTIAL && status != EBackupState::STATE_COMPLETED) {
|
2017-05-26 04:48:44 +08:00
|
|
|
throw backup_duplicate();
|
|
|
|
}
|
|
|
|
|
2019-03-05 07:58:36 +08:00
|
|
|
if (!g_network->isSimulated() && !forceAction) {
|
2020-01-23 07:41:22 +08:00
|
|
|
state StatusObject srcStatus = wait(StatusClient::statusFetcher(backupAgent->taskBucket->src));
|
|
|
|
StatusObject destStatus = wait(StatusClient::statusFetcher(dest));
|
2019-03-02 06:49:04 +08:00
|
|
|
checkAtomicSwitchOverConfig(srcStatus, destStatus, tagName);
|
|
|
|
}
|
2021-03-11 02:06:03 +08:00
|
|
|
|
2019-05-11 05:01:52 +08:00
|
|
|
state UID logUid = deterministicRandom()->randomUniqueID();
|
2017-05-26 04:48:44 +08:00
|
|
|
state Key logUidValue = BinaryWriter::toValue(logUid, Unversioned());
|
2017-06-01 07:23:37 +08:00
|
|
|
state UID logUidCurrent = wait(drAgent.getLogUid(backupAgent->taskBucket->src, tagName));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
if (logUidCurrent.isValid()) {
|
|
|
|
logUid = logUidCurrent;
|
|
|
|
logUidValue = BinaryWriter::toValue(logUid, Unversioned());
|
|
|
|
}
|
|
|
|
|
|
|
|
// Lock src, record commit version
|
|
|
|
state Transaction tr(backupAgent->taskBucket->src);
|
|
|
|
state Version commitVersion;
|
|
|
|
loop {
|
|
|
|
try {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(lockDatabase(&tr, logUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
tr.set(backupAgent->tagNames.pack(tagName), logUidValue);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr.commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
commitVersion = tr.getCommittedVersion();
|
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr.onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_SwitchoverLocked").detail("Version", commitVersion);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2017-09-08 08:41:20 +08:00
|
|
|
// Wait for the destination to apply mutations up to the lock commit before switching over.
|
2017-05-26 04:48:44 +08:00
|
|
|
state ReadYourWritesTransaction tr2(dest);
|
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
tr2.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr2.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Optional<Value> backupUid =
|
|
|
|
wait(tr2.get(backupAgent->states.get(BinaryWriter::toValue(destlogUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyFolderId)));
|
2019-03-19 06:03:43 +08:00
|
|
|
TraceEvent("DBA_SwitchoverBackupUID")
|
|
|
|
.detail("Uid", backupUid)
|
|
|
|
.detail("Key",
|
|
|
|
backupAgent->states.get(BinaryWriter::toValue(destlogUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyFolderId));
|
2017-05-26 04:48:44 +08:00
|
|
|
if (!backupUid.present())
|
|
|
|
throw backup_duplicate();
|
|
|
|
Optional<Value> v = wait(tr2.get(
|
|
|
|
BinaryWriter::toValue(destlogUid, Unversioned()).withPrefix(applyMutationsBeginRange.begin)));
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_SwitchoverVersion")
|
|
|
|
.detail("Version", v.present() ? BinaryReader::fromStringRef<Version>(v.get(), Unversioned()) : 0);
|
2017-05-26 04:48:44 +08:00
|
|
|
if (v.present() && BinaryReader::fromStringRef<Version>(v.get(), Unversioned()) >= commitVersion)
|
|
|
|
break;
|
|
|
|
|
|
|
|
state Future<Void> versionWatch = tr2.watch(
|
|
|
|
BinaryWriter::toValue(destlogUid, Unversioned()).withPrefix(applyMutationsBeginRange.begin));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr2.commit());
|
|
|
|
wait(versionWatch);
|
2017-05-26 04:48:44 +08:00
|
|
|
tr2.reset();
|
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr2.onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_SwitchoverReady").log();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
try {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(backupAgent->discontinueBackup(dest, tagName));
|
2017-05-26 04:48:44 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() != error_code_backup_duplicate && e.code() != error_code_backup_unneeded)
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2021-07-17 15:11:40 +08:00
|
|
|
wait(success(backupAgent->waitBackup(dest, tagName, StopWhenDone::True)));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_SwitchoverStopped").log();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2017-09-28 11:34:30 +08:00
|
|
|
state ReadYourWritesTransaction tr3(dest);
|
2017-09-08 08:41:20 +08:00
|
|
|
loop {
|
|
|
|
try {
|
2017-09-28 11:34:30 +08:00
|
|
|
tr3.setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr3.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
Version destVersion = wait(tr3.getReadVersion());
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_SwitchoverVersionUpgrade").detail("Src", commitVersion).detail("Dest", destVersion);
|
2017-09-08 08:41:20 +08:00
|
|
|
if (destVersion <= commitVersion) {
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Forcing dest backup cluster to higher version");
|
2017-09-28 11:34:30 +08:00
|
|
|
tr3.set(minRequiredCommitVersionKey, BinaryWriter::toValue(commitVersion + 1, Unversioned()));
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr3.commit());
|
2017-09-08 08:41:20 +08:00
|
|
|
} else {
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr3.onError(e));
|
2017-09-08 08:41:20 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_SwitchoverVersionUpgraded").log();
|
2017-09-08 08:41:20 +08:00
|
|
|
|
2018-04-19 10:37:24 +08:00
|
|
|
try {
|
2021-03-31 08:17:47 +08:00
|
|
|
wait(drAgent.submitBackup(backupAgent->taskBucket->src,
|
|
|
|
tagName,
|
|
|
|
backupRanges,
|
2021-07-17 15:11:40 +08:00
|
|
|
StopWhenDone::False,
|
2021-03-31 08:17:47 +08:00
|
|
|
addPrefix,
|
|
|
|
removePrefix,
|
2021-07-17 15:11:40 +08:00
|
|
|
LockDB::True,
|
2021-03-31 08:17:47 +08:00
|
|
|
DatabaseBackupAgent::PreBackupAction::NONE));
|
2018-04-19 10:37:24 +08:00
|
|
|
} catch (Error& e) {
|
|
|
|
if (e.code() != error_code_backup_duplicate)
|
|
|
|
throw;
|
|
|
|
}
|
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_SwitchoverSubmitted").log();
|
2018-04-19 10:37:24 +08:00
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(drAgent.waitSubmitted(backupAgent->taskBucket->src, tagName)));
|
2018-04-19 10:37:24 +08:00
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_SwitchoverStarted").log();
|
2018-04-19 10:37:24 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(backupAgent->unlockBackup(dest, tagName));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2021-07-27 10:55:10 +08:00
|
|
|
TraceEvent("DBA_SwitchoverUnlocked").log();
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR static Future<Void> discontinueBackup(DatabaseBackupAgent* backupAgent,
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Key tagName) {
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state UID logUid = wait(backupAgent->getLogUid(tr, tagName));
|
2020-09-24 05:50:34 +08:00
|
|
|
state EBackupState status = wait(backupAgent->getStateValue(tr, logUid));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_Discontinue").detail("Status", status);
|
2020-09-24 05:50:34 +08:00
|
|
|
if (!DatabaseBackupAgent::isRunnable(status)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
throw backup_unneeded();
|
|
|
|
}
|
|
|
|
|
|
|
|
state Optional<Value> stopWhenDoneValue =
|
|
|
|
wait(tr->get(backupAgent->config.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyConfigStopWhenDoneKey)));
|
|
|
|
|
|
|
|
if (stopWhenDoneValue.present()) {
|
|
|
|
throw backup_duplicate();
|
|
|
|
}
|
|
|
|
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
tr->set(backupAgent->config.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(BackupAgentBase::keyConfigStopWhenDoneKey),
|
|
|
|
StringRef());
|
|
|
|
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2020-10-21 13:48:53 +08:00
|
|
|
ACTOR static Future<Void> abortBackup(DatabaseBackupAgent* backupAgent,
|
|
|
|
Database cx,
|
|
|
|
Key tagName,
|
2021-07-05 09:30:55 +08:00
|
|
|
PartialBackup partial,
|
|
|
|
AbortOldBackup abortOldBackup,
|
|
|
|
DstOnly dstOnly,
|
|
|
|
WaitForDestUID waitForDestUID) {
|
2017-05-26 04:48:44 +08:00
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
2018-02-21 05:22:31 +08:00
|
|
|
state Key logUidValue, destUidValue;
|
|
|
|
state UID logUid, destUid;
|
2017-05-26 04:48:44 +08:00
|
|
|
state Value backupUid;
|
|
|
|
|
|
|
|
loop {
|
|
|
|
try {
|
2017-12-21 07:41:47 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
UID _logUid = wait(backupAgent->getLogUid(tr, tagName));
|
2018-02-21 05:22:31 +08:00
|
|
|
logUid = _logUid;
|
|
|
|
logUidValue = BinaryWriter::toValue(logUid, Unversioned());
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
state Future<EBackupState> statusFuture = backupAgent->getStateValue(tr, logUid);
|
2018-02-21 05:22:31 +08:00
|
|
|
state Future<UID> destUidFuture = backupAgent->getDestUid(tr, logUid);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(success(statusFuture) && success(destUidFuture));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
EBackupState status = statusFuture.get();
|
|
|
|
if (!backupAgent->isRunnable(status)) {
|
2017-05-26 04:48:44 +08:00
|
|
|
throw backup_unneeded();
|
|
|
|
}
|
2020-10-19 07:46:41 +08:00
|
|
|
UID destUid = destUidFuture.get();
|
|
|
|
if (destUid.isValid()) {
|
|
|
|
destUidValue = BinaryWriter::toValue(destUid, Unversioned());
|
2020-10-21 13:48:53 +08:00
|
|
|
} else if (destUidValue.size() == 0 && waitForDestUID) {
|
2020-10-19 07:46:41 +08:00
|
|
|
// Give DR task a chance to update destUid to avoid the problem of
|
2020-10-19 14:09:39 +08:00
|
|
|
// leftover version key. If we got an commit_unknown_result before,
|
|
|
|
// reuse the previous destUidValue.
|
2020-10-19 07:46:41 +08:00
|
|
|
throw not_committed();
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
Optional<Value> _backupUid =
|
|
|
|
wait(tr->get(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId)));
|
2017-05-26 04:48:44 +08:00
|
|
|
backupUid = _backupUid.get();
|
|
|
|
|
|
|
|
// Clearing the folder id will prevent future tasks from executing
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->clear(backupAgent->config.get(logUidValue).range());
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Clearing the end version of apply mutation cancels ongoing apply work
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->clear(logUidValue.withPrefix(applyMutationsEndRange.begin));
|
2017-09-19 07:56:34 +08:00
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->clear(prefixRange(logUidValue.withPrefix(applyLogKeys.begin)));
|
2017-12-01 09:37:00 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
tr->set(StringRef(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyStateStatus)),
|
|
|
|
StringRef(DatabaseBackupAgent::getStateText(EBackupState::STATE_PARTIALLY_ABORTED)));
|
2017-12-01 09:37:00 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_Abort").detail("CommitVersion", tr->getCommittedVersion());
|
2017-12-01 09:37:00 +08:00
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
2022-02-25 04:25:52 +08:00
|
|
|
TraceEvent("DBA_AbortError").errorUnsuppressed(e);
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-12-01 09:37:00 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
tr = makeReference<ReadYourWritesTransaction>(cx);
|
2017-12-01 09:37:00 +08:00
|
|
|
loop {
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
Fix VersionStamp problems by instead adding a COMMIT_ON_FIRST_PROXY transaction option.
Simulation identified the fact that we can violate the
VersionStamps-are-always-increasing promise via the following series of events:
1. On proxy 0, dumpData adds commit requests to proxy 0's commit promise stream
2. To any proxy, a client submits the first transaction of abortBackup, which stops further dumpData calls on proxy 0.
3. To any proxy that is not proxy 0, submit a transaction that checks if it needs to upgrade the destination version.
4. The transaction from (3) is committed
5. Transactions from (1) are committed
This is possible because the dumpData transactions have no read conflict
ranges, and thus it's impossible to make them abort due to "conflicting"
transactions. There's also no promise that if client C sends a commit to proxy
A, and later a client D sends a commit to proxy B, that B must log its commit
after A. (We only promise that if C is told it was committed before D is told
it was committed, then A committed before B.)
There was a failed attempt to fix this problem. We tried to add read conflict
ranges to dumpData transactions so that they could be aborted by "conflicting"
transactions. However, this failed because this now means that dumpData
transactions require conflict resolution, and the stale read version that they
use can cause them to be aborted with a transaction_too_old error.
(Transactions that don't have read conflict ranges will never return
transaction_too_old, because with no reads, the read snapshot version is
effectively meaningless.) This was never previously possible, so the existing
code doesn't retry commits, and to make things more complicated, the dumpData
commits must be applied in order. This would require either adding
dependencies to transactions (if A is going to commit then B must also be/have
committed), which would be complicated, or submitting transactions with a fixed
read version, and replaying the failed commits with a higher read version once
we get a transaction_too_old error, which would unacceptably slow down the
maximum throughput of dumpData.
Thus, we've instead elected to add a special transaction option that bypasses
proxy load balancing for commits, and always commits against proxy 0. We can
know for certain that after the transaction from (2) is committed, all of the
dumpData transactions that will be committed have been added to the commit
promise stream on proxy 0. Thus, if we enqueue another transaction against
proxy 0, we can know that it will be placed into the promise stream after all
of the dumpData transactions, thus providing the semantics that we require: no
dumpData transaction can commit after the destination version upgrade
transaction.
2017-12-20 08:44:07 +08:00
|
|
|
// dumpData's commits are unstoppable, and we need to make sure that no dumpData commits
|
|
|
|
// happen after this transaction, as it would mean that the applyMutationsBeginRange read we
|
|
|
|
// do isn't the final value, and thus a greater version of commits could have been applied.
|
|
|
|
// Thus, we need to commit it against the same proxy that all dumpData transactions were
|
|
|
|
// submitted to. The transaction above will stop any further dumpData calls from adding
|
|
|
|
// transactions to the proxy's commit promise stream, so our commit will come after all
|
|
|
|
// dumpData transactions.
|
|
|
|
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
|
2017-12-01 09:37:00 +08:00
|
|
|
try {
|
2017-09-19 07:56:34 +08:00
|
|
|
// Ensure that we're at a version higher than the data that we've written.
|
2018-02-21 05:22:31 +08:00
|
|
|
Optional<Value> lastApplied = wait(tr->get(logUidValue.withPrefix(applyMutationsBeginRange.begin)));
|
2017-09-19 07:56:34 +08:00
|
|
|
if (lastApplied.present()) {
|
2017-12-01 09:37:00 +08:00
|
|
|
Version current = tr->getReadVersion().get();
|
2017-09-19 07:56:34 +08:00
|
|
|
Version applied = BinaryReader::fromStringRef<Version>(lastApplied.get(), Unversioned());
|
2018-06-09 02:11:08 +08:00
|
|
|
TraceEvent("DBA_AbortVersionUpgrade").detail("Src", applied).detail("Dest", current);
|
2017-09-28 11:34:30 +08:00
|
|
|
if (current <= applied) {
|
2022-07-20 04:15:51 +08:00
|
|
|
CODE_PROBE(true, "Upgrading version of local database.");
|
2017-12-21 08:54:57 +08:00
|
|
|
// The +1 is because we want to make sure that a versionstamped operation can't reuse
|
|
|
|
// the same version as an already-applied transaction.
|
2017-09-19 07:56:34 +08:00
|
|
|
tr->set(minRequiredCommitVersionKey, BinaryWriter::toValue(applied + 1, Unversioned()));
|
Fix yet another VersionStamp DR bug.
In this episode, we discover that having a transaction retry loop in which the
transaction conditionally has write conflict ranges is potentially troublesome.
To simplify the problem, if we have two concurrent transaction loops:
retry {
if (rand() > .5) tr->set('x', rand());
if (rand() > .5) tr->set('y', rand());
}
and
retry {
x = tr->get('x')
y = tr->get('y')
if (x > y) {
tr->set('y', x)
}
tr->commit();
}
Is not guaranteed that x > y in the database after the second transaction
commits. This is because it could read an older snapshot of x and y, in which
x was greater than y, and thus not invoke set. This means that `tr` is now a
read-only transaction, which no-ops out of committing as an "optimization". If
we add any write conflict range to `tr`, it then will conflict checked and
committed, which would guarantee that x>y when it commits.
Replace the first transaction with dumpData, and the second with version
upgrade transaction, and you have the bug that we're fixing, why, and how.
2018-01-03 09:32:14 +08:00
|
|
|
} else {
|
|
|
|
// We need to enforce that the read we did of the applyMutationsBeginKey is the most
|
|
|
|
// recent and up to date value, as the proxy might have accepted a commit previously
|
|
|
|
// queued by dumpData after our read. Transactions that don't have write conflict ranges
|
|
|
|
// have a no-op commit(), as they become snapshot transactions to which we don't promise
|
|
|
|
// strict serializability. Therefore, we add an arbitrary write conflict range to
|
|
|
|
// request the strict serializability guarantee that is required.
|
|
|
|
tr->addWriteConflictRange(singleKeyRange(minRequiredCommitVersionKey));
|
2017-09-19 07:56:34 +08:00
|
|
|
}
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
break;
|
2017-12-01 09:37:00 +08:00
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
if (!dstOnly) {
|
|
|
|
state Future<Void> partialTimeout = partial ? delay(30.0) : Never();
|
|
|
|
state Reference<ReadYourWritesTransaction> srcTr(
|
|
|
|
new ReadYourWritesTransaction(backupAgent->taskBucket->src));
|
|
|
|
state Version beginVersion;
|
|
|
|
state Version endVersion;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
srcTr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
srcTr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Future<Optional<Value>> backupVersionF =
|
|
|
|
srcTr->get(backupAgent->sourceStates.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId));
|
|
|
|
wait(success(backupVersionF) || partialTimeout);
|
|
|
|
if (partialTimeout.isReady()) {
|
|
|
|
return Void();
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
if (backupVersionF.get().present() &&
|
|
|
|
BinaryReader::fromStringRef<Version>(backupVersionF.get().get(), Unversioned()) >
|
|
|
|
BinaryReader::fromStringRef<Version>(backupUid, Unversioned())) {
|
|
|
|
break;
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
if (abortOldBackup) {
|
2020-09-24 05:50:34 +08:00
|
|
|
srcTr->set(backupAgent->sourceStates.pack(DatabaseBackupAgent::keyStateStatus),
|
|
|
|
StringRef(BackupAgentBase::getStateText(EBackupState::STATE_ABORTED)));
|
2020-07-05 02:31:23 +08:00
|
|
|
srcTr->set(backupAgent->sourceStates.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId),
|
|
|
|
backupUid);
|
|
|
|
srcTr->clear(prefixRange(logUidValue.withPrefix(backupLogKeys.begin)));
|
|
|
|
srcTr->clear(prefixRange(logUidValue.withPrefix(logRangesRange.begin)));
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
|
|
|
|
Key latestVersionKey = logUidValue.withPrefix(destUidValue.withPrefix(backupLatestVersionsPrefix));
|
|
|
|
|
|
|
|
state Future<Optional<Key>> bVersionF = srcTr->get(latestVersionKey);
|
|
|
|
wait(success(bVersionF) || partialTimeout);
|
|
|
|
if (partialTimeout.isReady()) {
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
|
|
|
if (bVersionF.get().present()) {
|
|
|
|
beginVersion = BinaryReader::fromStringRef<Version>(bVersionF.get().get(), Unversioned());
|
|
|
|
} else {
|
|
|
|
break;
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
srcTr->set(backupAgent->sourceStates.pack(DatabaseBackupAgent::keyStateStatus),
|
|
|
|
StringRef(DatabaseBackupAgent::getStateText(EBackupState::STATE_PARTIALLY_ABORTED)));
|
2018-04-27 08:24:40 +08:00
|
|
|
srcTr->set(backupAgent->sourceStates.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId),
|
|
|
|
backupUid);
|
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
wait(eraseLogData(srcTr, logUidValue, destUidValue) || partialTimeout);
|
|
|
|
if (partialTimeout.isReady()) {
|
|
|
|
return Void();
|
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
wait(srcTr->commit() || partialTimeout);
|
|
|
|
if (partialTimeout.isReady()) {
|
|
|
|
return Void();
|
|
|
|
}
|
2019-09-28 09:32:27 +08:00
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
endVersion = srcTr->getCommittedVersion() + 1;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-07-05 02:31:23 +08:00
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
|
|
|
wait(srcTr->onError(e));
|
2019-09-28 09:32:27 +08:00
|
|
|
}
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2020-11-07 15:50:55 +08:00
|
|
|
tr = makeReference<ReadYourWritesTransaction>(cx);
|
2017-05-26 04:48:44 +08:00
|
|
|
loop {
|
|
|
|
try {
|
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
|
2018-02-21 05:22:31 +08:00
|
|
|
Optional<Value> v = wait(
|
|
|
|
tr->get(StringRef(backupAgent->config.get(logUidValue).pack(DatabaseBackupAgent::keyFolderId))));
|
2017-05-26 04:48:44 +08:00
|
|
|
if (v.present()) {
|
|
|
|
return Void();
|
|
|
|
}
|
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
tr->set(StringRef(backupAgent->states.get(logUidValue).pack(DatabaseBackupAgent::keyStateStatus)),
|
|
|
|
StringRef(DatabaseBackupAgent::getStateText(EBackupState::STATE_ABORTED)));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->commit());
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
return Void();
|
|
|
|
} catch (Error& e) {
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
ACTOR static Future<std::string> getStatus(DatabaseBackupAgent* backupAgent,
|
|
|
|
Database cx,
|
|
|
|
int errorLimit,
|
|
|
|
Key tagName) {
|
|
|
|
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state std::string statusText;
|
2020-02-26 07:48:00 +08:00
|
|
|
state int retries = 0;
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
loop {
|
|
|
|
try {
|
2022-09-17 07:44:31 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
|
2019-02-13 08:07:17 +08:00
|
|
|
wait(success(tr->getReadVersion())); // get the read version before getting a version from the source
|
|
|
|
// database to prevent the time differential from going negative
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
state Transaction scrTr(backupAgent->taskBucket->src);
|
|
|
|
scrTr.setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Future<Version> srcReadVersion = scrTr.getReadVersion();
|
|
|
|
|
|
|
|
statusText = "";
|
|
|
|
|
|
|
|
state UID logUid = wait(backupAgent->getLogUid(tr, tagName));
|
|
|
|
|
2017-12-15 05:54:01 +08:00
|
|
|
state Future<Optional<Value>> fPaused = tr->get(backupAgent->taskBucket->getPauseKey());
|
2021-05-04 04:14:16 +08:00
|
|
|
state Future<RangeResult> fErrorValues =
|
2020-02-26 07:48:00 +08:00
|
|
|
errorLimit > 0
|
|
|
|
? tr->getRange(backupAgent->errors.get(BinaryWriter::toValue(logUid, Unversioned())).range(),
|
|
|
|
errorLimit,
|
2021-07-17 15:11:40 +08:00
|
|
|
Snapshot::False,
|
|
|
|
Reverse::True)
|
2021-05-04 04:14:16 +08:00
|
|
|
: Future<RangeResult>();
|
2020-02-26 07:48:00 +08:00
|
|
|
state Future<Optional<Value>> fBackupUid =
|
|
|
|
tr->get(backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyFolderId));
|
|
|
|
state Future<Optional<Value>> fBackupVerison =
|
|
|
|
tr->get(BinaryWriter::toValue(logUid, Unversioned()).withPrefix(applyMutationsBeginRange.begin));
|
|
|
|
state Future<Optional<Key>> fTagName =
|
|
|
|
tr->get(backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(BackupAgentBase::keyConfigBackupTag));
|
|
|
|
state Future<Optional<Value>> fStopVersionKey =
|
|
|
|
tr->get(backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(BackupAgentBase::keyStateStop));
|
|
|
|
state Future<Optional<Key>> fBackupKeysPacked =
|
|
|
|
tr->get(backupAgent->config.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(BackupAgentBase::keyConfigBackupRanges));
|
2021-07-09 23:13:02 +08:00
|
|
|
state Future<Optional<Value>> flogVersionKey =
|
|
|
|
tr->get(backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(BackupAgentBase::keyStateLogBeginVersion));
|
2020-02-26 07:48:00 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
state EBackupState backupState = wait(backupAgent->getStateValue(tr, logUid));
|
|
|
|
|
|
|
|
if (backupState == EBackupState::STATE_NEVERRAN) {
|
2017-05-26 04:48:44 +08:00
|
|
|
statusText += "No previous backups found.\n";
|
2020-09-24 05:50:34 +08:00
|
|
|
} else {
|
2017-05-26 04:48:44 +08:00
|
|
|
state std::string tagNameDisplay;
|
2020-02-26 07:48:00 +08:00
|
|
|
Optional<Key> tagName = wait(fTagName);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Define the display tag name
|
|
|
|
if (tagName.present()) {
|
|
|
|
tagNameDisplay = tagName.get().toString();
|
|
|
|
}
|
|
|
|
|
2020-02-26 07:48:00 +08:00
|
|
|
state Optional<Value> stopVersionKey = wait(fStopVersionKey);
|
2021-07-09 23:13:02 +08:00
|
|
|
Optional<Value> logVersionKey = wait(flogVersionKey);
|
|
|
|
state std::string logVersionText =
|
|
|
|
". Last log version is " +
|
|
|
|
(logVersionKey.present()
|
|
|
|
? format("%lld", BinaryReader::fromStringRef<Version>(logVersionKey.get(), Unversioned()))
|
|
|
|
: "unset");
|
2020-02-26 07:48:00 +08:00
|
|
|
Optional<Key> backupKeysPacked = wait(fBackupKeysPacked);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-02-26 07:48:00 +08:00
|
|
|
state Standalone<VectorRef<KeyRangeRef>> backupRanges;
|
2017-05-26 04:48:44 +08:00
|
|
|
if (backupKeysPacked.present()) {
|
|
|
|
BinaryReader br(backupKeysPacked.get(), IncludeVersion());
|
|
|
|
br >> backupRanges;
|
|
|
|
}
|
|
|
|
|
|
|
|
switch (backupState) {
|
2020-09-24 05:50:34 +08:00
|
|
|
case EBackupState::STATE_SUBMITTED:
|
2017-05-26 04:48:44 +08:00
|
|
|
statusText += "The DR on tag `" + tagNameDisplay +
|
|
|
|
"' is NOT a complete copy of the primary database (just started).\n";
|
|
|
|
break;
|
2020-09-24 05:50:34 +08:00
|
|
|
case EBackupState::STATE_RUNNING:
|
2017-05-26 04:48:44 +08:00
|
|
|
statusText +=
|
|
|
|
"The DR on tag `" + tagNameDisplay + "' is NOT a complete copy of the primary database.\n";
|
|
|
|
break;
|
2020-09-24 05:50:34 +08:00
|
|
|
case EBackupState::STATE_RUNNING_DIFFERENTIAL:
|
2021-07-09 23:13:02 +08:00
|
|
|
statusText += "The DR on tag `" + tagNameDisplay +
|
|
|
|
"' is a complete copy of the primary database" + logVersionText + ".\n";
|
2017-05-26 04:48:44 +08:00
|
|
|
break;
|
2020-09-24 05:50:34 +08:00
|
|
|
case EBackupState::STATE_COMPLETED: {
|
2017-05-26 04:48:44 +08:00
|
|
|
Version stopVersion =
|
|
|
|
stopVersionKey.present()
|
|
|
|
? BinaryReader::fromStringRef<Version>(stopVersionKey.get(), Unversioned())
|
|
|
|
: -1;
|
|
|
|
statusText += "The previous DR on tag `" + tagNameDisplay + "' completed at version " +
|
|
|
|
format("%lld", stopVersion) + ".\n";
|
2020-09-24 05:50:34 +08:00
|
|
|
} break;
|
|
|
|
case EBackupState::STATE_PARTIALLY_ABORTED: {
|
2017-05-26 04:48:44 +08:00
|
|
|
statusText += "The previous DR on tag `" + tagNameDisplay + "' " +
|
2021-07-09 23:13:02 +08:00
|
|
|
BackupAgentBase::getStateText(backupState) + logVersionText + ".\n";
|
2017-05-26 04:48:44 +08:00
|
|
|
statusText += "Abort the DR with --cleanup before starting a new DR.\n";
|
|
|
|
break;
|
|
|
|
}
|
|
|
|
default:
|
|
|
|
statusText += "The previous DR on tag `" + tagNameDisplay + "' " +
|
2021-07-09 23:13:02 +08:00
|
|
|
BackupAgentBase::getStateText(backupState) + logVersionText + ".\n";
|
2017-05-26 04:48:44 +08:00
|
|
|
break;
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// Append the errors, if requested
|
|
|
|
if (errorLimit > 0) {
|
2021-05-04 04:14:16 +08:00
|
|
|
RangeResult values = wait(fErrorValues);
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
// Display the errors, if any
|
|
|
|
if (values.size() > 0) {
|
|
|
|
// Inform the user that the list of errors is complete or partial
|
|
|
|
statusText += (values.size() < errorLimit)
|
|
|
|
? "WARNING: Some DR agents have reported issues:\n"
|
|
|
|
: "WARNING: Some DR agents have reported issues (printing " +
|
|
|
|
std::to_string(errorLimit) + "):\n";
|
|
|
|
|
|
|
|
for (auto& s : values) {
|
|
|
|
statusText += " " + printable(s.value) + "\n";
|
|
|
|
}
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
// calculate time differential
|
2020-02-26 07:48:00 +08:00
|
|
|
Optional<Value> backupUid = wait(fBackupUid);
|
2017-05-26 04:48:44 +08:00
|
|
|
if (backupUid.present()) {
|
2020-02-26 07:48:00 +08:00
|
|
|
Optional<Value> v = wait(fBackupVerison);
|
2017-05-26 04:48:44 +08:00
|
|
|
if (v.present()) {
|
|
|
|
state Version destApplyBegin = BinaryReader::fromStringRef<Version>(v.get(), Unversioned());
|
|
|
|
Version sourceVersion = wait(srcReadVersion);
|
|
|
|
double secondsBehind =
|
|
|
|
((double)(sourceVersion - destApplyBegin)) / CLIENT_KNOBS->CORE_VERSIONSPERSECOND;
|
|
|
|
statusText += format("\nThe DR is %.6f seconds behind.\n", secondsBehind);
|
|
|
|
}
|
|
|
|
}
|
|
|
|
|
2017-12-15 05:54:01 +08:00
|
|
|
Optional<Value> paused = wait(fPaused);
|
|
|
|
if (paused.present()) {
|
|
|
|
statusText += format("\nAll DR agents have been paused.\n");
|
2017-10-31 03:35:00 +08:00
|
|
|
}
|
|
|
|
|
2017-05-26 04:48:44 +08:00
|
|
|
break;
|
|
|
|
} catch (Error& e) {
|
2020-02-26 07:48:00 +08:00
|
|
|
retries++;
|
|
|
|
if (retries > 5) {
|
|
|
|
statusText += format("\nWARNING: Could not fetch full DR status: %s\n", e.name());
|
|
|
|
return statusText;
|
|
|
|
}
|
2018-08-11 04:57:10 +08:00
|
|
|
wait(tr->onError(e));
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
}
|
|
|
|
|
|
|
|
return statusText;
|
|
|
|
}
|
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
ACTOR static Future<EBackupState> getStateValue(DatabaseBackupAgent* backupAgent,
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
UID logUid,
|
2021-07-03 12:41:50 +08:00
|
|
|
Snapshot snapshot) {
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Key statusKey = backupAgent->states.get(BinaryWriter::toValue(logUid, Unversioned()))
|
|
|
|
.pack(DatabaseBackupAgent::keyStateStatus);
|
2019-12-19 16:29:35 +08:00
|
|
|
Optional<Value> status = wait(tr->get(statusKey, snapshot));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
return (!status.present()) ? EBackupState::STATE_NEVERRAN : BackupAgentBase::getState(status.get().toString());
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2019-12-19 16:29:35 +08:00
|
|
|
ACTOR static Future<UID> getDestUid(DatabaseBackupAgent* backupAgent,
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
UID logUid,
|
2021-07-03 12:41:50 +08:00
|
|
|
Snapshot snapshot) {
|
2018-02-21 05:22:31 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
|
|
|
state Key destUidKey =
|
|
|
|
backupAgent->config.get(BinaryWriter::toValue(logUid, Unversioned())).pack(BackupAgentBase::destUid);
|
2019-12-19 16:29:35 +08:00
|
|
|
Optional<Value> destUid = wait(tr->get(destUidKey, snapshot));
|
2018-02-21 05:22:31 +08:00
|
|
|
|
|
|
|
return (destUid.present()) ? BinaryReader::fromStringRef<UID>(destUid.get(), Unversioned()) : UID();
|
|
|
|
}
|
|
|
|
|
2019-12-19 16:29:35 +08:00
|
|
|
ACTOR static Future<UID> getLogUid(DatabaseBackupAgent* backupAgent,
|
|
|
|
Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Key tagName,
|
2021-07-03 12:41:50 +08:00
|
|
|
Snapshot snapshot) {
|
2017-05-26 04:48:44 +08:00
|
|
|
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
|
|
|
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
2019-12-19 16:29:35 +08:00
|
|
|
state Optional<Value> logUid = wait(tr->get(backupAgent->tagNames.pack(tagName), snapshot));
|
2017-05-26 04:48:44 +08:00
|
|
|
|
|
|
|
return (logUid.present()) ? BinaryReader::fromStringRef<UID>(logUid.get(), Unversioned()) : UID();
|
|
|
|
}
|
|
|
|
};
|
|
|
|
|
|
|
|
Future<Void> DatabaseBackupAgent::unlockBackup(Reference<ReadYourWritesTransaction> tr, Key tagName) {
|
|
|
|
return DatabaseBackupAgentImpl::unlockBackup(this, tr, tagName);
|
|
|
|
}
|
|
|
|
|
2019-03-05 07:58:36 +08:00
|
|
|
Future<Void> DatabaseBackupAgent::atomicSwitchover(Database dest,
|
|
|
|
Key tagName,
|
|
|
|
Standalone<VectorRef<KeyRangeRef>> backupRanges,
|
|
|
|
Key addPrefix,
|
|
|
|
Key removePrefix,
|
2021-07-05 07:27:26 +08:00
|
|
|
ForceAction forceAction) {
|
2019-03-05 07:58:36 +08:00
|
|
|
return DatabaseBackupAgentImpl::atomicSwitchover(
|
|
|
|
this, dest, tagName, backupRanges, addPrefix, removePrefix, forceAction);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
Future<Void> DatabaseBackupAgent::submitBackup(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
Key tagName,
|
|
|
|
Standalone<VectorRef<KeyRangeRef>> backupRanges,
|
2021-07-05 07:27:26 +08:00
|
|
|
StopWhenDone stopWhenDone,
|
2017-05-26 04:48:44 +08:00
|
|
|
Key addPrefix,
|
|
|
|
Key removePrefix,
|
2021-07-05 07:27:26 +08:00
|
|
|
LockDB lockDatabase,
|
2021-03-31 08:17:47 +08:00
|
|
|
PreBackupAction backupAction) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return DatabaseBackupAgentImpl::submitBackup(
|
2021-03-31 08:17:47 +08:00
|
|
|
this, tr, tagName, backupRanges, stopWhenDone, addPrefix, removePrefix, lockDatabase, backupAction);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
Future<Void> DatabaseBackupAgent::discontinueBackup(Reference<ReadYourWritesTransaction> tr, Key tagName) {
|
|
|
|
return DatabaseBackupAgentImpl::discontinueBackup(this, tr, tagName);
|
|
|
|
}
|
|
|
|
|
2020-10-21 13:48:53 +08:00
|
|
|
Future<Void> DatabaseBackupAgent::abortBackup(Database cx,
|
|
|
|
Key tagName,
|
2021-07-05 09:30:55 +08:00
|
|
|
PartialBackup partial,
|
|
|
|
AbortOldBackup abortOldBackup,
|
|
|
|
DstOnly dstOnly,
|
|
|
|
WaitForDestUID waitForDestUID) {
|
2020-10-21 13:48:53 +08:00
|
|
|
return DatabaseBackupAgentImpl::abortBackup(this, cx, tagName, partial, abortOldBackup, dstOnly, waitForDestUID);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
|
|
|
Future<std::string> DatabaseBackupAgent::getStatus(Database cx, int errorLimit, Key tagName) {
|
|
|
|
return DatabaseBackupAgentImpl::getStatus(this, cx, errorLimit, tagName);
|
|
|
|
}
|
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
Future<EBackupState> DatabaseBackupAgent::getStateValue(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
UID logUid,
|
2021-07-03 12:41:50 +08:00
|
|
|
Snapshot snapshot) {
|
2019-12-19 16:29:35 +08:00
|
|
|
return DatabaseBackupAgentImpl::getStateValue(this, tr, logUid, snapshot);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2021-07-03 12:41:50 +08:00
|
|
|
Future<UID> DatabaseBackupAgent::getDestUid(Reference<ReadYourWritesTransaction> tr, UID logUid, Snapshot snapshot) {
|
2019-12-19 16:29:35 +08:00
|
|
|
return DatabaseBackupAgentImpl::getDestUid(this, tr, logUid, snapshot);
|
2018-02-21 05:22:31 +08:00
|
|
|
}
|
|
|
|
|
2021-07-03 12:41:50 +08:00
|
|
|
Future<UID> DatabaseBackupAgent::getLogUid(Reference<ReadYourWritesTransaction> tr, Key tagName, Snapshot snapshot) {
|
2019-12-19 16:29:35 +08:00
|
|
|
return DatabaseBackupAgentImpl::getLogUid(this, tr, tagName, snapshot);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2018-04-27 08:24:40 +08:00
|
|
|
Future<Void> DatabaseBackupAgent::waitUpgradeToLatestDrVersion(Database cx, Key tagName) {
|
|
|
|
return DatabaseBackupAgentImpl::waitUpgradeToLatestDrVersion(this, cx, tagName);
|
|
|
|
}
|
|
|
|
|
2021-07-05 07:27:26 +08:00
|
|
|
Future<EBackupState> DatabaseBackupAgent::waitBackup(Database cx, Key tagName, StopWhenDone stopWhenDone) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return DatabaseBackupAgentImpl::waitBackup(this, cx, tagName, stopWhenDone);
|
|
|
|
}
|
|
|
|
|
2020-09-24 05:50:34 +08:00
|
|
|
Future<EBackupState> DatabaseBackupAgent::waitSubmitted(Database cx, Key tagName) {
|
2017-05-26 04:48:44 +08:00
|
|
|
return DatabaseBackupAgentImpl::waitSubmitted(this, cx, tagName);
|
|
|
|
}
|
|
|
|
|
2019-12-19 16:29:35 +08:00
|
|
|
Future<int64_t> DatabaseBackupAgent::getRangeBytesWritten(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
UID logUid,
|
2021-07-03 12:41:50 +08:00
|
|
|
Snapshot snapshot) {
|
2019-12-19 16:29:35 +08:00
|
|
|
return DRConfig(logUid).rangeBytesWritten().getD(tr, snapshot);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|
|
|
|
|
2019-12-19 16:29:35 +08:00
|
|
|
Future<int64_t> DatabaseBackupAgent::getLogBytesWritten(Reference<ReadYourWritesTransaction> tr,
|
|
|
|
UID logUid,
|
2021-07-03 12:41:50 +08:00
|
|
|
Snapshot snapshot) {
|
2019-12-19 16:29:35 +08:00
|
|
|
return DRConfig(logUid).logBytesWritten().getD(tr, snapshot);
|
2017-05-26 04:48:44 +08:00
|
|
|
}
|