/* * BackupAgent.actor.h * * This source file is part of the FoundationDB open source project * * Copyright 2013-2018 Apple Inc. and the FoundationDB project authors * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. * You may obtain a copy of the License at * * http://www.apache.org/licenses/LICENSE-2.0 * * Unless required by applicable law or agreed to in writing, software * distributed under the License is distributed on an "AS IS" BASIS, * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. * See the License for the specific language governing permissions and * limitations under the License. */ #pragma once #if defined(NO_INTELLISENSE) && !defined(FDBCLIENT_BACKUP_AGENT_ACTOR_G_H) #define FDBCLIENT_BACKUP_AGENT_ACTOR_G_H #include "fdbclient/BackupAgent.actor.g.h" #elif !defined(FDBCLIENT_BACKUP_AGENT_ACTOR_H) #define FDBCLIENT_BACKUP_AGENT_ACTOR_H #include "flow/flow.h" #include "fdbclient/NativeAPI.actor.h" #include "fdbclient/TaskBucket.h" #include "fdbclient/Notified.h" #include "fdbrpc/IAsyncFile.h" #include "fdbclient/KeyBackedTypes.h" #include #include #include "fdbclient/BackupContainer.h" #include "flow/actorcompiler.h" // has to be last include class BackupAgentBase : NonCopyable { public: // Time formatter for anything backup or restore related static std::string formatTime(int64_t epochs); static int64_t parseTime(std::string timestamp); static std::string timeFormat() { return "YYYY/MM/DD.HH:MI:SS[+/-]HHMM"; } enum class EnumState { STATE_ERRORED = 0, STATE_SUBMITTED = 1, STATE_RUNNING = 2, STATE_RUNNING_DIFFERENTIAL = 3, STATE_COMPLETED = 4, STATE_NEVERRAN = 5, STATE_ABORTED = 6, STATE_PARTIALLY_ABORTED = 7 }; static const Key keyFolderId; static const Key keyBeginVersion; static const Key keyEndVersion; static const Key keyPrevBeginVersion; static const Key keyConfigBackupTag; static const Key keyConfigLogUid; static const Key keyConfigBackupRanges; static const Key keyConfigStopWhenDoneKey; static const Key keyStateStatus; static const Key keyStateStop; static const Key keyLastUid; static const Key keyBeginKey; static const Key keyEndKey; static const Key keyDrVersion; static const Key destUid; static const Key backupStartVersion; static const Key keyTagName; static const Key keyStates; static const Key keyConfig; static const Key keyErrors; static const Key keyRanges; static const Key keyTasks; static const Key keyFutures; static const Key keySourceStates; static const Key keySourceTagName; static const int logHeaderSize; // Convert the status text to an enumerated value static EnumState getState(std::string stateText) { auto enState = EnumState::STATE_ERRORED; if (stateText.empty()) { enState = EnumState::STATE_NEVERRAN; } else if (!stateText.compare("has been submitted")) { enState = EnumState::STATE_SUBMITTED; } else if (!stateText.compare("has been started")) { enState = EnumState::STATE_RUNNING; } else if (!stateText.compare("is differential")) { enState = EnumState::STATE_RUNNING_DIFFERENTIAL; } else if (!stateText.compare("has been completed")) { enState = EnumState::STATE_COMPLETED; } else if (!stateText.compare("has been aborted")) { enState = EnumState::STATE_ABORTED; } else if (!stateText.compare("has been partially aborted")) { enState = EnumState::STATE_PARTIALLY_ABORTED; } return enState; } // Convert the status enum to a text description static const char* getStateText(EnumState enState) { const char* stateText; switch (enState) { case EnumState::STATE_ERRORED: stateText = "has errored"; break; case EnumState::STATE_NEVERRAN: stateText = "has never been started"; break; case EnumState::STATE_SUBMITTED: stateText = "has been submitted"; break; case EnumState::STATE_RUNNING: stateText = "has been started"; break; case EnumState::STATE_RUNNING_DIFFERENTIAL: stateText = "is differential"; break; case EnumState::STATE_COMPLETED: stateText = "has been completed"; break; case EnumState::STATE_ABORTED: stateText = "has been aborted"; break; case EnumState::STATE_PARTIALLY_ABORTED: stateText = "has been partially aborted"; break; default: stateText = ""; break; } return stateText; } // Convert the status enum to a name static const char* getStateName(EnumState enState) { const char* s; switch (enState) { case EnumState::STATE_ERRORED: s = "Errored"; break; case EnumState::STATE_NEVERRAN: s = "NeverRan"; break; case EnumState::STATE_SUBMITTED: s = "Submitted"; break; case EnumState::STATE_RUNNING: s = "Running"; break; case EnumState::STATE_RUNNING_DIFFERENTIAL: s = "RunningDifferentially"; break; case EnumState::STATE_COMPLETED: s = "Completed"; break; case EnumState::STATE_ABORTED: s = "Aborted"; break; case EnumState::STATE_PARTIALLY_ABORTED: s = "Aborting"; break; default: s = ""; break; } return s; } // Determine if the specified state is runnable static bool isRunnable(EnumState enState) { bool isRunnable = false; switch (enState) { case EnumState::STATE_SUBMITTED: case EnumState::STATE_RUNNING: case EnumState::STATE_RUNNING_DIFFERENTIAL: case EnumState::STATE_PARTIALLY_ABORTED: isRunnable = true; break; default: break; } return isRunnable; } static const KeyRef getDefaultTag() { return StringRef(defaultTagName); } static const std::string getDefaultTagName() { return defaultTagName; } // This is only used for automatic backup name generation static Standalone getCurrentTime() { double t = now(); time_t curTime = t; char buffer[128]; struct tm* timeinfo; timeinfo = localtime(&curTime); strftime(buffer, 128, "%Y-%m-%d-%H-%M-%S", timeinfo); std::string time(buffer); return StringRef(time + format(".%06d", (int)(1e6*(t - curTime)))); } protected: static const std::string defaultTagName; }; class FileBackupAgent : public BackupAgentBase { public: FileBackupAgent(); FileBackupAgent(FileBackupAgent&& r) noexcept : subspace(std::move(r.subspace)), config(std::move(r.config)), lastRestorable(std::move(r.lastRestorable)), taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)) {} void operator=(FileBackupAgent&& r) noexcept { subspace = std::move(r.subspace); config = std::move(r.config); lastRestorable = std::move(r.lastRestorable), taskBucket = std::move(r.taskBucket); futureBucket = std::move(r.futureBucket); } KeyBackedProperty lastBackupTimestamp() { return config.pack(LiteralStringRef(__FUNCTION__)); } Future run(Database cx, double *pollDelay, int maxConcurrentTasks) { return taskBucket->run(cx, futureBucket, pollDelay, maxConcurrentTasks); } /** RESTORE **/ enum ERestoreState { UNITIALIZED = 0, QUEUED = 1, STARTING = 2, RUNNING = 3, COMPLETED = 4, ABORTED = 5 }; static StringRef restoreStateText(ERestoreState id); // parallel restore Future parallelRestoreFinish(Database cx, UID randomUID, bool unlockDB = true); Future submitParallelRestore(Database cx, Key backupTag, Standalone> backupRanges, Key bcUrl, Version targetVersion, bool lockDB, UID randomUID, Key addPrefix, Key removePrefix); Future atomicParallelRestore(Database cx, Key tagName, Standalone> ranges, Key addPrefix, Key removePrefix); // restore() will // - make sure that url is readable and appears to be a complete backup // - make sure the requested TargetVersion is valid // - submit a restore on the given tagName // - Optionally wait for the restore's completion. Will restore_error if restore fails or is aborted. // restore() will return the targetVersion which will be either the valid version passed in or the max restorable version for the given url. Future restore(Database cx, Optional cxOrig, Key tagName, Key url, Standalone> ranges, bool waitForComplete = true, Version targetVersion = -1, bool verbose = true, Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true, bool incrementalBackupOnly = false, Version beginVersion = -1); Future restore(Database cx, Optional cxOrig, Key tagName, Key url, bool waitForComplete = true, Version targetVersion = -1, bool verbose = true, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true, bool incrementalBackupOnly = false, Version beginVersion = -1) { Standalone> rangeRef; rangeRef.push_back_deep(rangeRef.arena(), range); return restore(cx, cxOrig, tagName, url, rangeRef, waitForComplete, targetVersion, verbose, addPrefix, removePrefix, lockDB, incrementalBackupOnly, beginVersion); } Future atomicRestore(Database cx, Key tagName, Standalone> ranges, Key addPrefix = Key(), Key removePrefix = Key()); Future atomicRestore(Database cx, Key tagName, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key()) { Standalone> rangeRef; rangeRef.push_back_deep(rangeRef.arena(), range); return atomicRestore(cx, tagName, rangeRef, addPrefix, removePrefix); } // Tries to abort the restore for a tag. Returns the final (stable) state of the tag. Future abortRestore(Reference tr, Key tagName); Future abortRestore(Database cx, Key tagName); // Waits for a restore tag to reach a final (stable) state. Future waitRestore(Database cx, Key tagName, bool verbose); // Get a string describing the status of a tag Future restoreStatus(Reference tr, Key tagName); Future restoreStatus(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr){ return restoreStatus(tr, tagName); }); } /** BACKUP METHODS **/ Future submitBackup(Reference tr, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone> backupRanges, bool stopWhenDone = true, bool partitionedLog = false, bool incrementalBackupOnly = false); Future submitBackup(Database cx, Key outContainer, int snapshotIntervalSeconds, std::string tagName, Standalone> backupRanges, bool stopWhenDone = true, bool partitionedLog = false, bool incrementalBackupOnly = false) { return runRYWTransactionFailIfLocked(cx, [=](Reference tr) { return submitBackup(tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone, partitionedLog, incrementalBackupOnly); }); } Future discontinueBackup(Reference tr, Key tagName); Future discontinueBackup(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr){ return discontinueBackup(tr, tagName); }); } // Terminate an ongoing backup, without waiting for the backup to finish. // Preconditions: // A backup is running with the tag of `tagName`. // Otherwise `backup_unneeded` will be thrown indicating that the backup never existed or already finished. // Postconditions: // No more tasks will be spawned to backup ranges of the database. // logRangesRange and backupLogKeys will be cleared for this backup. Future abortBackup(Reference tr, std::string tagName); Future abortBackup(Database cx, std::string tagName) { return runRYWTransaction(cx, [=](Reference tr){ return abortBackup(tr, tagName); }); } Future getStatus(Database cx, bool showErrors, std::string tagName); Future getStatusJSON(Database cx, std::string tagName); Future getLastRestorable(Reference tr, Key tagName, bool snapshot = false); void setLastRestorable(Reference tr, Key tagName, Version version); // stopWhenDone will return when the backup is stopped, if enabled. Otherwise, it // will return when the backup directory is restorable. Future waitBackup(Database cx, std::string tagName, bool stopWhenDone = true, Reference* pContainer = nullptr, UID* pUID = nullptr); static const Key keyLastRestorable; Future getTaskCount(Reference tr) { return taskBucket->getTaskCount(tr); } Future getTaskCount(Database cx) { return taskBucket->getTaskCount(cx); } Future watchTaskCount(Reference tr) { return taskBucket->watchTaskCount(tr); } Future checkActive(Database cx) { return taskBucket->checkActive(cx); } // If "pause" is true, pause all backups; otherwise, resume all. Future changePause(Database db, bool pause); friend class FileBackupAgentImpl; static const int dataFooterSize; Subspace subspace; Subspace config; Subspace lastRestorable; Reference taskBucket; Reference futureBucket; }; class DatabaseBackupAgent : public BackupAgentBase { public: DatabaseBackupAgent(); explicit DatabaseBackupAgent(Database src); DatabaseBackupAgent(DatabaseBackupAgent&& r) noexcept : subspace(std::move(r.subspace)), states(std::move(r.states)), config(std::move(r.config)), errors(std::move(r.errors)), ranges(std::move(r.ranges)), tagNames(std::move(r.tagNames)), taskBucket(std::move(r.taskBucket)), futureBucket(std::move(r.futureBucket)), sourceStates(std::move(r.sourceStates)), sourceTagNames(std::move(r.sourceTagNames)) {} void operator=(DatabaseBackupAgent&& r) noexcept { subspace = std::move(r.subspace); states = std::move(r.states); config = std::move(r.config); errors = std::move(r.errors); ranges = std::move(r.ranges); tagNames = std::move(r.tagNames); taskBucket = std::move(r.taskBucket); futureBucket = std::move(r.futureBucket); sourceStates = std::move(r.sourceStates); sourceTagNames = std::move(r.sourceTagNames); } Future run(Database cx, double *pollDelay, int maxConcurrentTasks) { return taskBucket->run(cx, futureBucket, pollDelay, maxConcurrentTasks); } Future atomicSwitchover(Database dest, Key tagName, Standalone> backupRanges, Key addPrefix, Key removePrefix, bool forceAction=false); Future unlockBackup(Reference tr, Key tagName); Future unlockBackup(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr){ return unlockBackup(tr, tagName); }); } Future submitBackup(Reference tr, Key tagName, Standalone> backupRanges, bool stopWhenDone = true, Key addPrefix = StringRef(), Key removePrefix = StringRef(), bool lockDatabase = false, bool databasesInSync=false); Future submitBackup(Database cx, Key tagName, Standalone> backupRanges, bool stopWhenDone = true, Key addPrefix = StringRef(), Key removePrefix = StringRef(), bool lockDatabase = false, bool databasesInSync=false) { return runRYWTransaction(cx, [=](Reference tr){ return submitBackup(tr, tagName, backupRanges, stopWhenDone, addPrefix, removePrefix, lockDatabase, databasesInSync); }); } Future discontinueBackup(Reference tr, Key tagName); Future discontinueBackup(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr){ return discontinueBackup(tr, tagName); }); } Future abortBackup(Database cx, Key tagName, bool partial = false, bool abortOldBackup = false, bool dstOnly = false); Future getStatus(Database cx, int errorLimit, Key tagName); Future getStateValue(Reference tr, UID logUid, bool snapshot = false); Future getStateValue(Database cx, UID logUid) { return runRYWTransaction(cx, [=](Reference tr){ return getStateValue(tr, logUid); }); } Future getDestUid(Reference tr, UID logUid, bool snapshot = false); Future getDestUid(Database cx, UID logUid) { return runRYWTransaction(cx, [=](Reference tr){ return getDestUid(tr, logUid); }); } Future getLogUid(Reference tr, Key tagName, bool snapshot = false); Future getLogUid(Database cx, Key tagName) { return runRYWTransaction(cx, [=](Reference tr){ return getLogUid(tr, tagName); }); } Future getRangeBytesWritten(Reference tr, UID logUid, bool snapshot = false); Future getLogBytesWritten(Reference tr, UID logUid, bool snapshot = false); // stopWhenDone will return when the backup is stopped, if enabled. Otherwise, it // will return when the backup directory is restorable. Future waitBackup(Database cx, Key tagName, bool stopWhenDone = true); Future waitSubmitted(Database cx, Key tagName); Future waitUpgradeToLatestDrVersion(Database cx, Key tagName); static const Key keyAddPrefix; static const Key keyRemovePrefix; static const Key keyRangeVersions; static const Key keyCopyStop; static const Key keyDatabasesInSync; static const int LATEST_DR_VERSION; Future getTaskCount(Reference tr) { return taskBucket->getTaskCount(tr); } Future getTaskCount(Database cx) { return taskBucket->getTaskCount(cx); } Future watchTaskCount(Reference tr) { return taskBucket->watchTaskCount(tr); } Future checkActive(Database cx) { return taskBucket->checkActive(cx); } friend class DatabaseBackupAgentImpl; Subspace subspace; Subspace states; Subspace config; Subspace errors; Subspace ranges; Subspace tagNames; Subspace sourceStates; Subspace sourceTagNames; Reference taskBucket; Reference futureBucket; }; typedef std::pair, Version> RangeResultWithVersion; struct RCGroup { Standalone items; Version version; uint64_t groupKey; RCGroup() : version(-1), groupKey(ULLONG_MAX) {}; template void serialize(Ar& ar) { serializer(ar, items, version, groupKey); } }; bool copyParameter(Reference source, Reference dest, Key key); Version getVersionFromString(std::string const& value); Standalone> getLogRanges(Version beginVersion, Version endVersion, Key destUidValue, int blockSize = CLIENT_KNOBS->LOG_RANGE_BLOCK_SIZE); Standalone> getApplyRanges(Version beginVersion, Version endVersion, Key backupUid); Future eraseLogData(Reference tr, Key logUidValue, Key destUidValue, Optional endVersion = Optional(), bool checkBackupUid = false, Version backupUid = 0); Key getApplyKey( Version version, Key backupUid ); Version getLogKeyVersion(Key key); std::pair decodeBKMutationLogKey(Key key); Future logError(Database cx, Key keyErrors, const std::string& message); Future logError(Reference tr, Key keyErrors, const std::string& message); Future checkVersion(Reference const& tr); ACTOR Future readCommitted(Database cx, PromiseStream results, Reference lock, KeyRangeRef range, bool terminator = true, bool systemAccess = false, bool lockAware = false); ACTOR Future readCommitted(Database cx, PromiseStream results, Future active, Reference lock, KeyRangeRef range, std::function(Key key)> groupBy, bool terminator = true, bool systemAccess = false, bool lockAware = false); ACTOR Future applyMutations(Database cx, Key uid, Key addPrefix, Key removePrefix, Version beginVersion, Version* endVersion, RequestStream commit, NotifiedVersion* committedVersion, Reference> keyVersion); ACTOR Future cleanupBackup(Database cx, bool deleteData); using EBackupState = BackupAgentBase::EnumState; template <> inline Tuple Codec::pack(EBackupState const& val) { return Tuple().append(static_cast(val)); } template <> inline EBackupState Codec::unpack(Tuple const& val) { return static_cast(val.getInt(0)); } // Key backed tags are a single-key slice of the TagUidMap, defined below. // The Value type of the key is a UidAndAbortedFlagT which is a pair of {UID, aborted_flag} // All tasks on the UID will have a validation key/value that requires aborted_flag to be // false, so changing that value, such as changing the UID or setting aborted_flag to true, // will kill all of the active tasks on that backup/restore UID. typedef std::pair UidAndAbortedFlagT; class KeyBackedTag : public KeyBackedProperty { public: KeyBackedTag() : KeyBackedProperty(StringRef()) {} KeyBackedTag(std::string tagName, StringRef tagMapPrefix); Future cancel(Reference tr) { std::string tag = tagName; Key _tagMapPrefix = tagMapPrefix; return map(get(tr), [tag, _tagMapPrefix, tr](Optional up) -> Void { if (up.present()) { // Set aborted flag to true up.get().second = true; KeyBackedTag(tag, _tagMapPrefix).set(tr, up.get()); } return Void(); }); } std::string tagName; Key tagMapPrefix; }; typedef KeyBackedMap TagMap; // Map of tagName to {UID, aborted_flag} located in the fileRestorePrefixRange keyspace. class TagUidMap : public KeyBackedMap { public: TagUidMap(const StringRef & prefix) : TagMap(LiteralStringRef("tag->uid/").withPrefix(prefix)), prefix(prefix) {} ACTOR static Future> getAll_impl(TagUidMap* tagsMap, Reference tr, bool snapshot); Future> getAll(Reference tr, bool snapshot = false) { return getAll_impl(this, tr, snapshot); } Key prefix; }; static inline KeyBackedTag makeRestoreTag(std::string tagName) { return KeyBackedTag(tagName, fileRestorePrefixRange.begin); } static inline KeyBackedTag makeBackupTag(std::string tagName) { return KeyBackedTag(tagName, fileBackupPrefixRange.begin); } static inline Future> getAllRestoreTags(Reference tr, bool snapshot = false) { return TagUidMap(fileRestorePrefixRange.begin).getAll(tr, snapshot); } static inline Future> getAllBackupTags(Reference tr, bool snapshot = false) { return TagUidMap(fileBackupPrefixRange.begin).getAll(tr, snapshot); } class KeyBackedConfig { public: static struct { static TaskParam uid() {return LiteralStringRef(__FUNCTION__); } } TaskParams; KeyBackedConfig(StringRef prefix, UID uid = UID()) : uid(uid), prefix(prefix), configSpace(uidPrefixKey(LiteralStringRef("uid->config/").withPrefix(prefix), uid)) {} KeyBackedConfig(StringRef prefix, Reference task) : KeyBackedConfig(prefix, TaskParams.uid().get(task)) {} Future toTask(Reference tr, Reference task, bool setValidation = true) { // Set the uid task parameter TaskParams.uid().set(task, uid); if (!setValidation) { return Void(); } // Set the validation condition for the task which is that the restore uid's tag's uid is the same as the restore uid. // Get this uid's tag, then get the KEY for the tag's uid but don't read it. That becomes the validation key // which TaskBucket will check, and its value must be this restore config's uid. UID u = uid; // 'this' could be invalid in lambda Key p = prefix; return map(tag().get(tr), [u,p,task](Optional const &tag) -> Void { if(!tag.present()) throw restore_error(); // Validation contition is that the uidPair key must be exactly {u, false} TaskBucket::setValidationCondition(task, KeyBackedTag(tag.get(), p).key, Codec::pack({u, false}).pack()); return Void(); }); } KeyBackedProperty tag() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } UID getUid() { return uid; } Key getUidAsKey() { return BinaryWriter::toValue(uid, Unversioned()); } void clear(Reference tr) { tr->clear(configSpace.range()); } // lastError is a pair of error message and timestamp expressed as an int64_t KeyBackedProperty> lastError() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedMap> lastErrorPerType() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Updates the error per type map and the last error property Future updateErrorInfo(Database cx, Error e, std::string message) { // Avoid capture of this ptr auto © = *this; return runRYWTransaction(cx, [=] (Reference tr) mutable { tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::LOCK_AWARE); return map(tr->getReadVersion(), [=] (Version v) mutable { copy.lastError().set(tr, {message, v}); copy.lastErrorPerType().set(tr, e.code(), {message, v}); return Void(); }); }); } protected: UID uid; Key prefix; Subspace configSpace; }; template<> inline Tuple Codec>::pack(Reference const &bc) { return Tuple().append(StringRef(bc->getURL())); } template<> inline Reference Codec>::unpack(Tuple const &val) { return IBackupContainer::openContainer(val.getString(0).toString()); } class BackupConfig : public KeyBackedConfig { public: BackupConfig(UID uid = UID()) : KeyBackedConfig(fileBackupPrefixRange.begin, uid) {} BackupConfig(Reference task) : KeyBackedConfig(fileBackupPrefixRange.begin, task) {} // rangeFileMap maps a keyrange file's End to its Begin and Filename struct RangeSlice { Key begin; Version version; std::string fileName; int64_t fileSize; Tuple pack() const { return Tuple().append(begin).append(version).append(StringRef(fileName)).append(fileSize); } static RangeSlice unpack(Tuple const &t) { RangeSlice r; int i = 0; r.begin = t.getString(i++); r.version = t.getInt(i++); r.fileName = t.getString(i++).toString(); r.fileSize = t.getInt(i++); return r; } }; // Map of range end boundaries to info about the backup file written for that range. typedef KeyBackedMap RangeFileMapT; RangeFileMapT snapshotRangeFileMap() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Number of kv range files that were both committed to persistent storage AND inserted into // the snapshotRangeFileMap. Note that since insertions could replace 1 or more existing // map entries this is not necessarily the number of entries currently in the map. // This value exists to help with sizing of kv range folders for BackupContainers that // require it. KeyBackedBinaryValue snapshotRangeFileCount() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Coalesced set of ranges already dispatched for writing. typedef KeyBackedMap RangeDispatchMapT; RangeDispatchMapT snapshotRangeDispatchMap() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Interval to use for determining the target end version for new snapshots KeyBackedProperty snapshotIntervalSeconds() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // When the current snapshot began KeyBackedProperty snapshotBeginVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // When the current snapshot is desired to end. // This can be changed at runtime to speed up or slow down a snapshot KeyBackedProperty snapshotTargetEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotBatchSize() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotBatchFuture() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotBatchDispatchDoneKey() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotDispatchLastShardsBehind() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty snapshotDispatchLastVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } Future initNewSnapshot(Reference tr, int64_t intervalSeconds = -1) { BackupConfig © = *this; // Capture this by value instead of this ptr Future beginVersion = tr->getReadVersion(); Future defaultInterval = 0; if(intervalSeconds < 0) defaultInterval = copy.snapshotIntervalSeconds().getOrThrow(tr); // Make sure read version and possibly the snapshot interval value are ready, then clear/init the snapshot config members return map(success(beginVersion) && success(defaultInterval), [=](Void) mutable { copy.snapshotRangeFileMap().clear(tr); copy.snapshotRangeDispatchMap().clear(tr); copy.snapshotBatchSize().clear(tr); copy.snapshotBatchFuture().clear(tr); copy.snapshotBatchDispatchDoneKey().clear(tr); if(intervalSeconds < 0) intervalSeconds = defaultInterval.get(); Version endVersion = beginVersion.get() + intervalSeconds * CLIENT_KNOBS->CORE_VERSIONSPERSECOND; copy.snapshotBeginVersion().set(tr, beginVersion.get()); copy.snapshotTargetEndVersion().set(tr, endVersion); copy.snapshotRangeFileCount().set(tr, 0); copy.snapshotDispatchLastVersion().clear(tr); copy.snapshotDispatchLastShardsBehind().clear(tr); return Void(); }); } KeyBackedBinaryValue rangeBytesWritten() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedBinaryValue logBytesWritten() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty stateEnum() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty> backupContainer() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true when all backup workers for saving mutation logs have been started. KeyBackedProperty allWorkerStarted() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Each backup worker adds its (epoch, tag.id) to this property. KeyBackedProperty>> startedBackupWorkers() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true if backup worker is enabled. KeyBackedProperty backupWorkerEnabled() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true if partitioned log is enabled (only useful if backup worker is also enabled). KeyBackedProperty partitionedLogEnabled() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Set to true if only requesting incremental backup without base snapshot. KeyBackedProperty incrementalBackupOnly() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Latest version for which all prior versions have saved by backup workers. KeyBackedProperty latestBackupWorkerSavedVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Stop differntial logging if already started or don't start after completing KV ranges KeyBackedProperty stopWhenDone() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // Latest version for which all prior versions have had their log copy tasks completed KeyBackedProperty latestLogEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // The end version of the last complete snapshot KeyBackedProperty latestSnapshotEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } // The end version of the first complete snapshot KeyBackedProperty firstSnapshotEndVersion() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } KeyBackedProperty destUidValue() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } Future> getLatestRestorableVersion(Reference tr) { tr->setOption(FDBTransactionOptions::READ_SYSTEM_KEYS); tr->setOption(FDBTransactionOptions::READ_LOCK_AWARE); auto lastLog = latestLogEndVersion().get(tr); auto firstSnapshot = firstSnapshotEndVersion().get(tr); auto workerEnabled = backupWorkerEnabled().get(tr); auto plogEnabled = partitionedLogEnabled().get(tr); auto workerVersion = latestBackupWorkerSavedVersion().get(tr); auto incrementalBackup = incrementalBackupOnly().get(tr); return map(success(lastLog) && success(firstSnapshot) && success(workerEnabled) && success(plogEnabled) && success(workerVersion) && success(incrementalBackup), [=](Void) -> Optional { // The latest log greater than the oldest snapshot is the restorable version Optional logVersion = workerEnabled.get().present() && workerEnabled.get().get() && plogEnabled.get().present() && plogEnabled.get().get() ? workerVersion.get() : lastLog.get(); if (logVersion.present() && firstSnapshot.get().present() && logVersion.get() > firstSnapshot.get().get()) { return std::max(logVersion.get() - 1, firstSnapshot.get().get()); } if (logVersion.present() && incrementalBackup.isReady() && incrementalBackup.get().present() && incrementalBackup.get().get()) { return logVersion.get() - 1; } return {}; }); } KeyBackedProperty> backupRanges() { return configSpace.pack(LiteralStringRef(__FUNCTION__)); } void startMutationLogs(Reference tr, KeyRangeRef backupRange, Key destUidValue) { Key mutationLogsDestKey = destUidValue.withPrefix(backupLogKeys.begin); tr->set(logRangesEncodeKey(backupRange.begin, BinaryReader::fromStringRef(destUidValue, Unversioned())), logRangesEncodeValue(backupRange.end, mutationLogsDestKey)); } Future logError(Database cx, Error e, std::string details, void *taskInstance = nullptr) { if(!uid.isValid()) { TraceEvent(SevError, "FileBackupErrorNoUID").error(e).detail("Description", details); return Void(); } TraceEvent t(SevWarn, "FileBackupError"); t.error(e).detail("BackupUID", uid).detail("Description", details).detail("TaskInstance", (uint64_t)taskInstance); // key_not_found could happen if(e.code() == error_code_key_not_found) t.backtrace(); return updateErrorInfo(cx, e, details); } }; // Helper class for reading restore data from a buffer and throwing the right errors. struct StringRefReader { StringRefReader(StringRef s = StringRef(), Error e = Error()) : rptr(s.begin()), end(s.end()), failure_error(e) {} // Return remainder of data as a StringRef StringRef remainder() { return StringRef(rptr, end - rptr); } // Return a pointer to len bytes at the current read position and advance read pos const uint8_t* consume(unsigned int len) { if (rptr == end && len != 0) throw end_of_stream(); const uint8_t* p = rptr; rptr += len; if (rptr > end) throw failure_error; return p; } // Return a T from the current read position and advance read pos template const T consume() { return *(const T*)consume(sizeof(T)); } // Functions for consuming big endian (network byte order) integers. // Consumes a big endian number, swaps it to little endian, and returns it. int32_t consumeNetworkInt32() { return (int32_t)bigEndian32((uint32_t)consume()); } uint32_t consumeNetworkUInt32() { return bigEndian32(consume()); } // Convert big Endian value (e.g., encoded in log file) into a littleEndian uint64_t value. int64_t consumeNetworkInt64() { return (int64_t)bigEndian64((uint32_t)consume()); } uint64_t consumeNetworkUInt64() { return bigEndian64(consume()); } bool eof() { return rptr == end; } const uint8_t *rptr, *end; Error failure_error; }; namespace fileBackup { ACTOR Future>> decodeRangeFileBlock(Reference file, int64_t offset, int len); // Return a block of contiguous padding bytes "\0xff" for backup files, growing if needed. Value makePadding(int size); } // For fast restore simulation test // For testing addPrefix feature in fast restore. // Transform db content in restoreRanges by removePrefix and then addPrefix. // Assume: DB is locked ACTOR Future transformRestoredDatabase(Database cx, Standalone> backupRanges, Key addPrefix, Key removePrefix); void simulateBlobFailure(); #include "flow/unactorcompiler.h" #endif